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/10/04 20:01:34 UTC

[04/46] lucenenet git commit: Finished port of Facet.Taxonomy.WriterCache.TestCompactLabelToOrdinal and fixed bugs with FacetLabel hash code, CategoryPathUtils.EqualsToSerialized(), CharBlockArray.SubSequence() and CompactLabelToOrdinal.Open() to make th

Finished port of Facet.Taxonomy.WriterCache.TestCompactLabelToOrdinal and fixed bugs with FacetLabel hash code, CategoryPathUtils.EqualsToSerialized(), CharBlockArray.SubSequence() and CompactLabelToOrdinal.Open() to make the tests pass. Added an extra non-random test to make debugging easier.


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

Branch: refs/heads/master
Commit: 08dfc1bd0e84712eed5d4d34dd629408998cd196
Parents: 861aa73
Author: Shad Storhaug <sh...@shadstorhaug.com>
Authored: Sat Sep 24 16:32:20 2016 +0700
Committer: Shad Storhaug <sh...@shadstorhaug.com>
Committed: Mon Oct 3 23:30:38 2016 +0700

----------------------------------------------------------------------
 src/Lucene.Net.Core/Support/StreamUtils.cs      |  21 +--
 src/Lucene.Net.Facet/Taxonomy/FacetLabel.cs     |   7 +-
 .../Taxonomy/WriterCache/CategoryPathUtils.cs   |   9 +-
 .../Taxonomy/WriterCache/CharBlockArray.cs      |  12 +-
 .../WriterCache/CompactLabelToOrdinal.cs        |  16 +-
 .../Taxonomy/WriterCache/TestCharBlockArray.cs  |   4 +-
 .../WriterCache/TestCompactLabelToOrdinal.cs    | 183 ++++++++++++++++---
 7 files changed, 189 insertions(+), 63 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucenenet/blob/08dfc1bd/src/Lucene.Net.Core/Support/StreamUtils.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Core/Support/StreamUtils.cs b/src/Lucene.Net.Core/Support/StreamUtils.cs
index 7e05717..e6cd549 100644
--- a/src/Lucene.Net.Core/Support/StreamUtils.cs
+++ b/src/Lucene.Net.Core/Support/StreamUtils.cs
@@ -1,11 +1,5 @@
-\ufeffusing System;
-using System.Collections.Generic;
-using System.IO;
-using System.Linq;
-using System.Runtime.Serialization;
+\ufeffusing System.IO;
 using System.Runtime.Serialization.Formatters.Binary;
-using System.Text;
-using System.Threading.Tasks;
 
 namespace Lucene.Net.Support
 {
@@ -15,24 +9,23 @@ namespace Lucene.Net.Support
 
         public static void SerializeToStream(object o, Stream outputStream)
         {
-            // LUCENENET TODO: It would probably be better to serialize to
-            // XML so this works across .NET framework versions or alternatively
-            // find/create an alternative binary formatter implementation that works that way.
             Formatter.Serialize(outputStream, o);
         }
 
+        public static void SerializeToStream(object o, BinaryWriter writer)
+        {
+            Formatter.Serialize(writer.BaseStream, o);
+        }
+
         public static object DeserializeFromStream(Stream stream)
         {
-            stream.Seek(0, SeekOrigin.Begin);
             object o = Formatter.Deserialize(stream);
             return o;
         }
 
         public static object DeserializeFromStream(BinaryReader reader)
         {
-            var stream = reader.BaseStream;
-            stream.Seek(0, SeekOrigin.Begin);
-            object o = Formatter.Deserialize(stream);
+            object o = Formatter.Deserialize(reader.BaseStream);
             return o;
         }
     }

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/08dfc1bd/src/Lucene.Net.Facet/Taxonomy/FacetLabel.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Facet/Taxonomy/FacetLabel.cs b/src/Lucene.Net.Facet/Taxonomy/FacetLabel.cs
index a448d08..9a6884c 100644
--- a/src/Lucene.Net.Facet/Taxonomy/FacetLabel.cs
+++ b/src/Lucene.Net.Facet/Taxonomy/FacetLabel.cs
@@ -162,7 +162,12 @@ namespace Lucene.Net.Facet.Taxonomy
                 return 0;
             }
 
-            return Arrays.GetHashCode(Components);
+            int hash = Length;
+            for (int i = 0; i < Length; i++)
+            {
+                hash = hash * 31 + Components[i].GetHashCode();
+            }
+            return hash;
         }
 
         /// <summary>

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/08dfc1bd/src/Lucene.Net.Facet/Taxonomy/WriterCache/CategoryPathUtils.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Facet/Taxonomy/WriterCache/CategoryPathUtils.cs b/src/Lucene.Net.Facet/Taxonomy/WriterCache/CategoryPathUtils.cs
index 24b0fa2..eb63f56 100644
--- a/src/Lucene.Net.Facet/Taxonomy/WriterCache/CategoryPathUtils.cs
+++ b/src/Lucene.Net.Facet/Taxonomy/WriterCache/CategoryPathUtils.cs
@@ -1,6 +1,7 @@
-\ufeffnamespace Lucene.Net.Facet.Taxonomy.WriterCache
-{
+\ufeffusing System;
 
+namespace Lucene.Net.Facet.Taxonomy.WriterCache
+{
     /*
      * Licensed to the Apache Software Foundation (ASF) under one or more
      * contributor license agreements.  See the NOTICE file distributed with
@@ -85,7 +86,7 @@
                     return false;
                 }
 
-                if (!cp.Components[i].Equals(charBlockArray.SubSequence(offset, offset + len)))
+                if (!cp.Components[i].Equals(charBlockArray.SubSequence(offset, offset + len), StringComparison.Ordinal))
                 {
                     return false;
                 }
@@ -93,7 +94,5 @@
             }
             return true;
         }
-
     }
-
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/08dfc1bd/src/Lucene.Net.Facet/Taxonomy/WriterCache/CharBlockArray.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Facet/Taxonomy/WriterCache/CharBlockArray.cs b/src/Lucene.Net.Facet/Taxonomy/WriterCache/CharBlockArray.cs
index a38329b..c1c9825 100644
--- a/src/Lucene.Net.Facet/Taxonomy/WriterCache/CharBlockArray.cs
+++ b/src/Lucene.Net.Facet/Taxonomy/WriterCache/CharBlockArray.cs
@@ -163,7 +163,7 @@ namespace Lucene.Net.Facet.Taxonomy.WriterCache
                 {
                     toCopy = remainingInBlock;
                 }
-                s.CopyTo(offset, this.current.chars, this.current.length, offset + toCopy - offset);
+                s.CopyTo(offset, this.current.chars, this.current.length, toCopy);
                 offset += toCopy;
                 remain -= toCopy;
                 this.current.length += toCopy;
@@ -187,7 +187,7 @@ namespace Lucene.Net.Facet.Taxonomy.WriterCache
             return b.chars[IndexInBlock(index)];
         }
 
-        public ICharSequence SubSequence(int start, int end)
+        public string SubSequence(int start, int end)
         {
             int remaining = end - start;
             StringBuilder sb = new StringBuilder(remaining);
@@ -201,11 +201,13 @@ namespace Lucene.Net.Facet.Taxonomy.WriterCache
                 remaining -= numToAppend;
                 indexInBlock = 0; // 2nd+ iterations read from start of the block
             }
-            return new StringCharSequenceWrapper(sb.ToString());
+            return sb.ToString();
         }
 
-
-
+        ICharSequence ICharSequence.SubSequence(int start, int end)
+        {
+            return new StringCharSequenceWrapper(this.SubSequence(start, end));
+        }
 
         public override string ToString()
         {

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/08dfc1bd/src/Lucene.Net.Facet/Taxonomy/WriterCache/CompactLabelToOrdinal.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Facet/Taxonomy/WriterCache/CompactLabelToOrdinal.cs b/src/Lucene.Net.Facet/Taxonomy/WriterCache/CompactLabelToOrdinal.cs
index f717fb1..d0cebe2 100644
--- a/src/Lucene.Net.Facet/Taxonomy/WriterCache/CompactLabelToOrdinal.cs
+++ b/src/Lucene.Net.Facet/Taxonomy/WriterCache/CompactLabelToOrdinal.cs
@@ -1,11 +1,9 @@
 \ufeffusing System;
-using System.Collections.Generic;
 using System.IO;
-using Lucene.Net.Store;
+using System.Runtime.Serialization;
 
 namespace Lucene.Net.Facet.Taxonomy.WriterCache
 {
-
     /*
      * Licensed to the Apache Software Foundation (ASF) under one or more
      * contributor license agreements.  See the NOTICE file distributed with
@@ -23,7 +21,6 @@ namespace Lucene.Net.Facet.Taxonomy.WriterCache
      * limitations under the License.
      */
 
-
     /// <summary>
     /// This is a very efficient LabelToOrdinal implementation that uses a
     /// CharBlockArray to store all labels and a configurable number of HashArrays to
@@ -396,7 +393,7 @@ namespace Lucene.Net.Facet.Taxonomy.WriterCache
         /// Opens the file and reloads the CompactLabelToOrdinal. The file it expects
         /// is generated from the <seealso cref="#flush(File)"/> command.
         /// </summary>
-        internal static CompactLabelToOrdinal Open(string file, float loadFactor, int numHashArrays)
+        internal static CompactLabelToOrdinal Open(FileInfo file, float loadFactor, int numHashArrays)
         {
             /// <summary>
             /// Part of the file is the labelRepository, which needs to be rehashed
@@ -411,7 +408,7 @@ namespace Lucene.Net.Facet.Taxonomy.WriterCache
             BinaryReader dis = null;
             try
             {
-                dis = new BinaryReader(new FileStream(file,FileMode.Open,FileAccess.Read));
+                dis = new BinaryReader(new FileStream(file.FullName, FileMode.Open, FileAccess.Read));
 
                 // TaxiReader needs to load the "counter" or occupancy (L2O) to know
                 // the next unique facet. we used to load the delimiter too, but
@@ -465,9 +462,9 @@ namespace Lucene.Net.Facet.Taxonomy.WriterCache
                 }
 
             }
-            catch (DllNotFoundException)
+            catch (SerializationException se)
             {
-                throw new IOException("Invalid file format. Cannot deserialize.");
+                throw new IOException("Invalid file format. Cannot deserialize.", se);
             }
             finally
             {
@@ -482,7 +479,7 @@ namespace Lucene.Net.Facet.Taxonomy.WriterCache
 
         }
 
-        internal virtual void Flush(FileStream stream)
+        internal virtual void Flush(Stream stream)
         {
             using (BinaryWriter dos = new BinaryWriter(stream))
             {
@@ -508,5 +505,4 @@ namespace Lucene.Net.Facet.Taxonomy.WriterCache
             }
         }
     }
-
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/08dfc1bd/src/Lucene.Net.Tests.Facet/Taxonomy/WriterCache/TestCharBlockArray.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Tests.Facet/Taxonomy/WriterCache/TestCharBlockArray.cs b/src/Lucene.Net.Tests.Facet/Taxonomy/WriterCache/TestCharBlockArray.cs
index f059a79..a763f80 100644
--- a/src/Lucene.Net.Tests.Facet/Taxonomy/WriterCache/TestCharBlockArray.cs
+++ b/src/Lucene.Net.Tests.Facet/Taxonomy/WriterCache/TestCharBlockArray.cs
@@ -76,13 +76,13 @@ namespace Lucene.Net.Facet.Taxonomy.WriterCache
 
             DirectoryInfo tempDir = CreateTempDir("growingchararray");
             FileInfo f = new FileInfo(Path.Combine(tempDir.FullName, "GrowingCharArrayTest.tmp"));
-            using (Stream @out = new FileStream(f.FullName, FileMode.OpenOrCreate, FileAccess.Write))
+            using (var @out = new FileStream(f.FullName, FileMode.OpenOrCreate, FileAccess.Write))
             {
                 array.Flush(@out);
                 @out.Flush();
             }
 
-            using (Stream @in = new FileStream(f.FullName, FileMode.Open, FileAccess.Read))
+            using (var @in = new FileStream(f.FullName, FileMode.Open, FileAccess.Read))
             {
                 array = CharBlockArray.Open(@in);
                 AssertEqualsInternal("GrowingCharArray<->StringBuilder mismatch after flush/load.", builder, array);

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/08dfc1bd/src/Lucene.Net.Tests.Facet/Taxonomy/WriterCache/TestCompactLabelToOrdinal.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Tests.Facet/Taxonomy/WriterCache/TestCompactLabelToOrdinal.cs b/src/Lucene.Net.Tests.Facet/Taxonomy/WriterCache/TestCompactLabelToOrdinal.cs
index 549bf09..eafdd62 100644
--- a/src/Lucene.Net.Tests.Facet/Taxonomy/WriterCache/TestCompactLabelToOrdinal.cs
+++ b/src/Lucene.Net.Tests.Facet/Taxonomy/WriterCache/TestCompactLabelToOrdinal.cs
@@ -1,14 +1,12 @@
-\ufeffusing System;
+\ufeffusing NUnit.Framework;
+using System;
 using System.Collections.Generic;
-using Lucene.Net.Support;
-using NUnit.Framework;
+using System.IO;
+using System.Text;
+using System.Text.RegularExpressions;
 
 namespace Lucene.Net.Facet.Taxonomy.WriterCache
 {
-
-
-    using TestUtil = Lucene.Net.Util.TestUtil;
-
     /*
      * Licensed to the Apache Software Foundation (ASF) under one or more
      * contributor license agreements.  See the NOTICE file distributed with
@@ -25,11 +23,11 @@ namespace Lucene.Net.Facet.Taxonomy.WriterCache
      * See the License for the specific language governing permissions and
      * limitations under the License.
      */
+
     [TestFixture]
     public class TestCompactLabelToOrdinal : FacetTestCase
     {
-        /* not finished to porting yet because of missing decoder implementation */
-        /*
+        [Test]
         public virtual void TestL2O()
         {
             LabelToOrdinal map = new LabelToOrdinalMap();
@@ -43,18 +41,17 @@ namespace Lucene.Net.Facet.Taxonomy.WriterCache
             byte[] buffer = new byte[50];
 
             Random random = Random();
-            for (int i = 0; i < numUniqueValues; )
+            for (int i = 0; i < numUniqueValues;)
             {
                 random.NextBytes(buffer);
                 int size = 1 + random.Next(buffer.Length);
 
                 // This test is turning random bytes into a string,
                 // this is asking for trouble.
-                CharsetDecoder decoder = StandardCharsets.UTF_8.newDecoder().onUnmappableCharacter(CodingErrorAction.REPLACE).onMalformedInput(CodingErrorAction.REPLACE);
-                uniqueValues[i] = decoder.decode(ByteBuffer.Wrap(buffer, 0, size)).ToString();
+                uniqueValues[i] = Encoding.UTF8.GetString(buffer, 0, size);
                 // we cannot have empty path components, so eliminate all prefix as well
                 // as middle consecutive delimiter chars.
-                uniqueValues[i] = uniqueValues[i].replaceAll("/+", "/");
+                uniqueValues[i] = Regex.Replace(uniqueValues[i], "/+", "/");
                 if (uniqueValues[i].StartsWith("/", StringComparison.Ordinal))
                 {
                     uniqueValues[i] = uniqueValues[i].Substring(1);
@@ -66,16 +63,21 @@ namespace Lucene.Net.Facet.Taxonomy.WriterCache
             }
 
             var tmpDir = CreateTempDir("testLableToOrdinal");
-            var f = new File(tmpDir, "CompactLabelToOrdinalTest.tmp");
+            var f = new FileInfo(Path.Combine(tmpDir.FullName, "CompactLabelToOrdinalTest.tmp"));
             int flushInterval = 10;
 
             for (int i = 0; i < n; i++)
             {
                 if (i > 0 && i % flushInterval == 0)
                 {
-                    compact.Flush(f);
-                    compact = CompactLabelToOrdinal.open(f, 0.15f, 3);
-                    Assert.True(f.delete());
+                    using (var fileStream = new FileStream(f.FullName, FileMode.OpenOrCreate, FileAccess.ReadWrite))
+                    {
+                        compact.Flush(fileStream);
+                    }
+                    compact = CompactLabelToOrdinal.Open(f, 0.15f, 3);
+                    //assertTrue(f.Delete());
+                    f.Delete();
+                    assertFalse(File.Exists(f.FullName));
                     if (flushInterval < (n / 10))
                     {
                         flushInterval *= 10;
@@ -97,7 +99,12 @@ namespace Lucene.Net.Facet.Taxonomy.WriterCache
                 int ord1 = map.GetOrdinal(label);
                 int ord2 = compact.GetOrdinal(label);
 
-                Assert.AreEqual(ord1, ord2);
+                if (VERBOSE)
+                {
+                    Console.WriteLine("Testing label: " + label.ToString());
+                }
+
+                assertEquals(ord1, ord2);
 
                 if (ord1 == LabelToOrdinal.INVALID_ORDINAL)
                 {
@@ -121,13 +128,136 @@ namespace Lucene.Net.Facet.Taxonomy.WriterCache
                 }
                 int ord1 = map.GetOrdinal(label);
                 int ord2 = compact.GetOrdinal(label);
-                Assert.AreEqual(ord1, ord2);
+
+                if (VERBOSE)
+                {
+                    Console.WriteLine("Testing label 2: " + label.ToString());
+                }
+
+                assertEquals(ord1, ord2);
+            }
+        }
+
+        /// <summary>
+        /// LUCENENET specific test similar to TestL2O without any randomness, useful for debugging
+        /// </summary>
+        [Test]
+        public virtual void TestL2OBasic()
+        {
+            LabelToOrdinal map = new LabelToOrdinalMap();
+
+            CompactLabelToOrdinal compact = new CompactLabelToOrdinal(200, 0.15f, 3);
+
+            int n = 50;
+
+            string[] uniqueValues = new string[]
+            {
+                @"\ufffd",
+                @"\ufffdr\ufffdG\ufffd\ufffdF\ufffd\u0382\ufffd7\u0019\ufffdh\ufffd\u0015\ufffd\ufffd\ufffd#\u001d3\r{\ufffd\ufffdq\ufffd_\ufffd\ufffd\ufffd\u0502\ufffd\ufffd\ufffd\ufffd\ufffd\ufffd",
+                "foo bar one",
+                new string(new char[] { (char)65533, (char)65533, (char)65, (char)65533, (char)45, (char)106, (char)40, (char)643, (char)65533, (char)11, (char)65533, (char)88, (char)65533, (char)78, (char)126, (char)56, (char)12, (char)71 }),
+                "foo bar two",
+                "foo bar three",
+                "foo bar four",
+                "foo bar five",
+                "foo bar six",
+                "foo bar seven",
+                "foo bar eight",
+                "foo bar nine",
+                "foo bar ten",
+                "foo/bar/one",
+                "foo/bar/two",
+                "foo/bar/three",
+                "foo/bar/four",
+                "foo/bar/five",
+                "foo/bar/six",
+                "foo/bar/seven",
+                "foo/bar/eight",
+                "foo/bar/nine",
+                "foo/bar/ten",
+                ""
+            };
+
+            var tmpDir = CreateTempDir("testLableToOrdinal");
+            var f = new FileInfo(Path.Combine(tmpDir.FullName, "CompactLabelToOrdinalTest.tmp"));
+            int flushInterval = 10;
+
+            for (int i = 0; i < n; i++)
+            {
+                if (i > 0 && i % flushInterval == 0)
+                {
+                    using (var fileStream = new FileStream(f.FullName, FileMode.OpenOrCreate, FileAccess.ReadWrite))
+                    {
+                        compact.Flush(fileStream);
+                    }
+                    compact = CompactLabelToOrdinal.Open(f, 0.15f, 3);
+                    //assertTrue(f.Delete());
+                    f.Delete();
+                    assertFalse(File.Exists(f.FullName));
+                    if (flushInterval < (n / 10))
+                    {
+                        flushInterval *= 10;
+                    }
+                }
+
+                FacetLabel label = new FacetLabel();
+                foreach (string s in uniqueValues)
+                {
+                    if (s.Length == 0)
+                    {
+                        label = new FacetLabel();
+                    }
+                    else
+                    {
+                        label = new FacetLabel(s.Split("/".ToCharArray(), StringSplitOptions.RemoveEmptyEntries));
+                    }
+
+                    int ord1 = map.GetOrdinal(label);
+                    int ord2 = compact.GetOrdinal(label);
+
+                    if (VERBOSE)
+                    {
+                        Console.WriteLine("Testing label: " + label.ToString());
+                    }
+
+                    assertEquals(ord1, ord2);
+
+                    if (ord1 == LabelToOrdinal.INVALID_ORDINAL)
+                    {
+                        ord1 = compact.NextOrdinal;
+                        map.AddLabel(label, ord1);
+                        compact.AddLabel(label, ord1);
+                    }
+                }
+            }
+
+            for (int i = 0; i < uniqueValues.Length; i++)
+            {
+                FacetLabel label;
+                string s = uniqueValues[i];
+                if (s.Length == 0)
+                {
+                    label = new FacetLabel();
+                }
+                else
+                {
+                    label = new FacetLabel(s.Split("/".ToCharArray(), StringSplitOptions.RemoveEmptyEntries));
+                }
+                int ord1 = map.GetOrdinal(label);
+                int ord2 = compact.GetOrdinal(label);
+
+                if (VERBOSE)
+                {
+                    Console.WriteLine("Testing label 2: " + label.ToString());
+                }
+
+                assertEquals(ord1, ord2);
             }
         }
 
         private class LabelToOrdinalMap : LabelToOrdinal
         {
-            internal IDictionary<FacetLabel, int?> map = new Dictionary<FacetLabel, int?>();
+            internal IDictionary<FacetLabel, int> map = new Dictionary<FacetLabel, int>();
 
             internal LabelToOrdinalMap()
             {
@@ -140,12 +270,13 @@ namespace Lucene.Net.Facet.Taxonomy.WriterCache
 
             public override int GetOrdinal(FacetLabel label)
             {
-                int? value = map[label];
-                return (value != null) ? (int)value : LabelToOrdinal.INVALID_ORDINAL;
+                int value;
+                if (map.TryGetValue(label, out value))
+                {
+                    return value;
+                }
+                return LabelToOrdinal.INVALID_ORDINAL;
             }
-
-        } */
-
+        } 
     }
-
 }
\ No newline at end of file