You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucenenet.apache.org by ni...@apache.org on 2017/02/02 12:42:43 UTC

[12/14] lucenenet git commit: Lucene.Net.Analysis.Common: Reverted ThaiTokenizer, ThaiWordFilter, and SegmentingTokenizerBase back to their original API and used IcuBreakIterator as the backing BreakIterator.

Lucene.Net.Analysis.Common: Reverted ThaiTokenizer, ThaiWordFilter, and SegmentingTokenizerBase back to their original API and used IcuBreakIterator as the backing BreakIterator.


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

Branch: refs/heads/api-work
Commit: 506f55a64a9d82e1965da077cc38c64ecd9214eb
Parents: 5a7cb17
Author: Shad Storhaug <sh...@shadstorhaug.com>
Authored: Thu Feb 2 16:30:07 2017 +0700
Committer: Shad Storhaug <sh...@shadstorhaug.com>
Committed: Thu Feb 2 18:30:36 2017 +0700

----------------------------------------------------------------------
 .../Analysis/Th/ThaiTokenizer.cs                | 126 ++++++-------------
 .../Analysis/Th/ThaiWordFilter.cs               |   6 +-
 .../Analysis/Util/SegmentingTokenizerBase.cs    |  91 +++++---------
 .../Analysis/Th/TestThaiAnalyzer.cs             |   9 --
 .../Util/TestSegmentingTokenizerBase.cs         |  15 +--
 5 files changed, 76 insertions(+), 171 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucenenet/blob/506f55a6/src/Lucene.Net.Analysis.Common/Analysis/Th/ThaiTokenizer.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Analysis.Common/Analysis/Th/ThaiTokenizer.cs b/src/Lucene.Net.Analysis.Common/Analysis/Th/ThaiTokenizer.cs
index 52f6750..ae3ab1a 100644
--- a/src/Lucene.Net.Analysis.Common/Analysis/Th/ThaiTokenizer.cs
+++ b/src/Lucene.Net.Analysis.Common/Analysis/Th/ThaiTokenizer.cs
@@ -1,13 +1,13 @@
-\ufeffusing Icu;
+\ufeff//using Icu;
 using Lucene.Net.Analysis.TokenAttributes;
 using Lucene.Net.Analysis.Util;
 using Lucene.Net.Support;
 using System;
 using System.Collections.Generic;
+using System.Globalization;
 using System.IO;
 using System.Linq;
 using System.Text.RegularExpressions;
-using BreakIterator = Icu.BreakIterator;
 
 namespace Lucene.Net.Analysis.Th
 {
@@ -43,17 +43,12 @@ namespace Lucene.Net.Analysis.Th
         /// If this is false, this tokenizer will not work at all!
         /// </summary>
         public static readonly bool DBBI_AVAILABLE;
-        
-        //LUCENENET: Specifying to use Thai locale.
-        private static readonly Locale LocaleThai = new Locale("th");
-
-        private static readonly IEnumerable<Boundary> proto;
+        private static readonly BreakIterator proto = new IcuBreakIterator(Icu.BreakIterator.UBreakIteratorType.WORD, new CultureInfo("th"));
         static ThaiTokenizer()
         {
             // check that we have a working dictionary-based break iterator for thai
-            proto = BreakIterator.GetWordBoundaries(LocaleThai, "\u0e20\u0e32\u0e29\u0e32\u0e44\u0e17\u0e22", includeSpacesAndPunctuation: false).ToArray();
-            var first = proto.FirstOrDefault();
-            DBBI_AVAILABLE = first != default(Boundary) && first.End == 4;
+            proto.SetText("\u0e20\u0e32\u0e29\u0e32\u0e44\u0e17\u0e22");
+            DBBI_AVAILABLE = proto.IsBoundary(4);
         }
 
         private readonly ThaiWordBreaker wordBreaker;
@@ -75,18 +70,18 @@ namespace Lucene.Net.Analysis.Th
         /// <summary>
         /// Creates a new ThaiTokenizer, supplying the AttributeFactory </summary>
         public ThaiTokenizer(AttributeFactory factory, TextReader reader)
-              : base(factory, reader, LocaleThai, BreakIterator.UBreakIteratorType.SENTENCE)
+              : base(factory, reader, new IcuBreakIterator(Icu.BreakIterator.UBreakIteratorType.SENTENCE, new CultureInfo("th")))
         {
             if (!DBBI_AVAILABLE)
             {
                 throw new System.NotSupportedException("This JRE does not have support for Thai segmentation");
             }
-            wordBreaker = new ThaiWordBreaker(LocaleUS);
+            wordBreaker = new ThaiWordBreaker(new IcuBreakIterator(Icu.BreakIterator.UBreakIteratorType.WORD, CultureInfo.InvariantCulture));
             termAtt = AddAttribute<ICharTermAttribute>();
             offsetAtt = AddAttribute<IOffsetAttribute>();
         }
 
-        protected internal override void SetNextSentence(int sentenceStart, int sentenceEnd)
+        protected override void SetNextSentence(int sentenceStart, int sentenceEnd)
         {
             this.sentenceStart = sentenceStart;
             this.sentenceEnd = sentenceEnd;
@@ -94,7 +89,7 @@ namespace Lucene.Net.Analysis.Th
             wordBreaker.SetText(new string(wrapper.Text, wrapper.Start, wrapper.Length));
         }
 
-        protected internal override bool IncrementWord()
+        protected override bool IncrementWord()
         {
             int start = wordBreaker.Current();
             if (start == BreakIterator.DONE)
@@ -104,7 +99,7 @@ namespace Lucene.Net.Analysis.Th
 
             // find the next set of boundaries, skipping over non-tokens
             int end = wordBreaker.Next();
-            while (end != BreakIterator.DONE && !char.IsLetterOrDigit((char)Support.Character.CodePointAt(m_buffer, sentenceStart + start, sentenceEnd)))
+            while (end != BreakIterator.DONE && !char.IsLetterOrDigit((char)Character.CodePointAt(m_buffer, sentenceStart + start, sentenceEnd)))
             {
                 start = end;
                 end = wordBreaker.Next();
@@ -126,137 +121,86 @@ namespace Lucene.Net.Analysis.Th
     /// LUCENENET specific class to patch the behavior of the ICU BreakIterator.
     /// Corrects the breaking of words by finding transitions between Thai and non-Thai
     /// characters.
-    /// 
-    /// This logic assumes that the Java BreakIterator also breaks up Thai numerals from
-    /// Arabic numerals (1, 2, 3, etc.). That is, it assumes the first test below passes
-    /// and the second test fails in Lucene (not attempted).
-    /// 
-    /// ThaiAnalyzer analyzer = new ThaiAnalyzer(TEST_VERSION_CURRENT, CharArraySet.EMPTY_SET);
-    /// AssertAnalyzesTo(analyzer, "\u0e51\u0e52\u0e53456", new string[] { "\u0e51\u0e52\u0e53", "456" });
-    /// AssertAnalyzesTo(analyzer, "\u0e51\u0e52\u0e53456", new string[] { "\u0e51\u0e52\u0e53456" });
     /// </summary>
     internal class ThaiWordBreaker
     {
-        private readonly Locale locale;
-        private IEnumerator<Boundary> wordBreaker;
-        private int currentIndex;
+        private readonly BreakIterator wordBreaker;
         private string text;
         private readonly IList<int> transitions = new List<int>();
         private readonly static Regex thaiPattern = new Regex(@"\p{IsThai}", RegexOptions.Compiled | RegexOptions.CultureInvariant);
 
-        public ThaiWordBreaker(Locale locale)
+        public ThaiWordBreaker(BreakIterator wordBreaker)
         {
-            if (locale == null)
+            if (wordBreaker == null)
             {
-                throw new ArgumentNullException("locale");
+                throw new ArgumentNullException("wordBreaker");
             }
-
-            this.locale = locale;
-            currentIndex = int.MinValue;
+            this.wordBreaker = wordBreaker;
         }
 
         public void SetText(string text)
         {
             this.text = text;
-            wordBreaker = BreakIterator.GetWordBoundaries(locale, text, includeSpacesAndPunctuation: false).ToList().GetEnumerator();
-            currentIndex = wordBreaker.MoveNext()
-                ? wordBreaker.Current.Start : BreakIterator.DONE;
+            wordBreaker.SetText(text);
         }
 
         public int Current()
         {
-            return currentIndex;
+            if (transitions.Any())
+            {
+                return transitions.First();
+            }
+            return wordBreaker.Current;
         }
 
         public int Next()
         {
-            // Tracking whether a transition was returned last time
-            // next was called. If that is the case, and there are no
-            // transitions left, then we return the End index in the
-            // wordbreaker.Current
-            bool transitionReturned = false;
-
             if (transitions.Any())
             {
-                transitionReturned = currentIndex == transitions[0];
                 transitions.RemoveAt(0);
             }
-
             if (transitions.Any())
             {
-                currentIndex = transitions.First();
-                return currentIndex;
-            }
-            else if (transitionReturned)
-            {
-                currentIndex = wordBreaker.Current.End;
+                return transitions.First();
             }
-
             return GetNext();
         }
 
         private int GetNext()
         {
-            bool isThaiLetter = false, isNonThaiLetter = false;
+            bool isThai = false, isNonThai = false;
             bool prevWasThai = false, prevWasNonThai = false;
+            int prev = wordBreaker.Current;
+            int current = wordBreaker.Next();
 
-            int previous = currentIndex;
-            int current;
-
-            if (currentIndex == wordBreaker.Current.Start)
-            {
-                current = wordBreaker.Current.End;
-            }
-            else if (wordBreaker.MoveNext())
-            {
-                // The break iterator works by returning the start and end
-                // boundary of each word it finds. Consider the two words,
-                // 
-                if (currentIndex == wordBreaker.Current.Start)
-                {
-                    current = wordBreaker.Current.End;
-                }
-                else
-                {
-                    current = wordBreaker.Current.Start;
-                }
-            }
-            else
-            {
-                current = BreakIterator.DONE;
-            }
-
-            if (current != BreakIterator.DONE && current - previous > 0)
+            if (current != BreakIterator.DONE && current - prev > 0)
             {
                 // Find all of the transitions between Thai and non-Thai characters and digits
-                for (int i = previous; i < current; i++)
+                for (int i = prev; i < current; i++)
                 {
                     char c = text[i];
-                    isThaiLetter = char.IsLetter(c) && thaiPattern.IsMatch(c.ToString());
-                    isNonThaiLetter = char.IsLetter(c) && !isThaiLetter;
+                    isThai = char.IsLetter(c) && thaiPattern.IsMatch(c.ToString());
+                    isNonThai = char.IsLetter(c) && !isThai;
 
-                    if ((prevWasThai && isNonThaiLetter) ||
-                        (prevWasNonThai && isThaiLetter))
+                    if ((prevWasThai && isNonThai) ||
+                        (prevWasNonThai && isThai))
                     {
                         transitions.Add(i);
                     }
 
                     // record the values for comparison with the next loop
-                    prevWasThai = isThaiLetter;
-                    prevWasNonThai = isNonThaiLetter;
+                    prevWasThai = isThai;
+                    prevWasNonThai = isNonThai;
                 }
 
                 if (transitions.Any())
                 {
                     transitions.Add(current);
-                    currentIndex = transitions.First();
-
-                    return currentIndex;
+                    return transitions.First();
                 }
             }
 
-            currentIndex = current;
-            return currentIndex;
+            return current;
         }
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/506f55a6/src/Lucene.Net.Analysis.Common/Analysis/Th/ThaiWordFilter.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Analysis.Common/Analysis/Th/ThaiWordFilter.cs b/src/Lucene.Net.Analysis.Common/Analysis/Th/ThaiWordFilter.cs
index 2b21033..d55733a 100644
--- a/src/Lucene.Net.Analysis.Common/Analysis/Th/ThaiWordFilter.cs
+++ b/src/Lucene.Net.Analysis.Common/Analysis/Th/ThaiWordFilter.cs
@@ -1,9 +1,11 @@
-\ufeffusing Icu;
+\ufeff//using Icu;
 using Lucene.Net.Analysis.Core;
 using Lucene.Net.Analysis.TokenAttributes;
 using Lucene.Net.Analysis.Util;
+using Lucene.Net.Support;
 using Lucene.Net.Util;
 using System;
+using System.Globalization;
 using System.Text.RegularExpressions;
 
 namespace Lucene.Net.Analysis.Th
@@ -48,7 +50,7 @@ namespace Lucene.Net.Analysis.Th
         /// If this is false, this filter will not work at all!
         /// </summary>
         public static readonly bool DBBI_AVAILABLE = ThaiTokenizer.DBBI_AVAILABLE;
-        private readonly ThaiWordBreaker breaker = new ThaiWordBreaker(new Locale());
+        private readonly ThaiWordBreaker breaker = new ThaiWordBreaker(new IcuBreakIterator(Icu.BreakIterator.UBreakIteratorType.WORD, new CultureInfo("th")));
         private readonly CharArrayIterator charIterator = CharArrayIterator.NewWordInstance();
 
         private readonly bool handlePosIncr;

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/506f55a6/src/Lucene.Net.Analysis.Common/Analysis/Util/SegmentingTokenizerBase.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Analysis.Common/Analysis/Util/SegmentingTokenizerBase.cs b/src/Lucene.Net.Analysis.Common/Analysis/Util/SegmentingTokenizerBase.cs
index 502b7da..ca0b994 100644
--- a/src/Lucene.Net.Analysis.Common/Analysis/Util/SegmentingTokenizerBase.cs
+++ b/src/Lucene.Net.Analysis.Common/Analysis/Util/SegmentingTokenizerBase.cs
@@ -1,13 +1,8 @@
-\ufeffusing System;
-using System.Collections;
-using System.Collections.Generic;
+\ufeffusing Lucene.Net.Analysis.TokenAttributes;
+using Lucene.Net.Support;
+using System;
 using System.Diagnostics;
-using System.Linq;
 using System.IO;
-using Icu;
-using Lucene.Net.Analysis.TokenAttributes;
-using Version = Lucene.Net.Util.LuceneVersion;
-
 
 namespace Lucene.Net.Analysis.Util
 {
@@ -28,9 +23,8 @@ namespace Lucene.Net.Analysis.Util
      * limitations under the License.
      */
 
-
     /// <summary>
-    /// Breaks text into sentences with a <seealso cref="BreakIterator"/> and
+    /// Breaks text into sentences with a <see cref="BreakIterator"/> and
     /// allows subclasses to decompose these sentences into words.
     /// <para>
     /// This can be used by subclasses that need sentence context 
@@ -46,11 +40,8 @@ namespace Lucene.Net.Analysis.Util
     /// </summary>
     public abstract class SegmentingTokenizerBase : Tokenizer
     {
-        // LUCENENET: Using Icu .NET to get Local_US
-        public static readonly Locale LocaleUS = new Locale("en-US");
-
-        protected internal const int BUFFERMAX = 1024;
-        protected internal readonly char[] m_buffer = new char[BUFFERMAX];
+        protected const int BUFFERMAX = 1024;
+        protected readonly char[] m_buffer = new char[BUFFERMAX];
         /// <summary>
         /// true length of text in the buffer </summary>
         private int length = 0;
@@ -59,11 +50,9 @@ namespace Lucene.Net.Analysis.Util
         private int usableLength = 0;
         /// <summary>
         /// accumulated offset of previous buffers for this reader, for offsetAtt </summary>
-        protected internal int m_offset = 0;
+        protected int m_offset = 0;
 
-        private readonly Locale locale;
-        private readonly BreakIterator.UBreakIteratorType iteratorType;
-        private IEnumerator<Boundary> enumerator;
+        private readonly BreakIterator iterator;
         private readonly CharArrayIterator wrapper = CharArrayIterator.NewSentenceInstance();
 
         private readonly IOffsetAttribute offsetAtt;
@@ -77,28 +66,19 @@ namespace Lucene.Net.Analysis.Util
         /// be provided to this constructor.
         /// </para>
         /// </summary>
-        protected SegmentingTokenizerBase(TextReader reader, BreakIterator.UBreakIteratorType iteratorType)
-            : this(AttributeFactory.DEFAULT_ATTRIBUTE_FACTORY, reader, LocaleUS, iteratorType)
-        { }
-
-        protected SegmentingTokenizerBase(TextReader reader, Locale locale, BreakIterator.UBreakIteratorType iteratorType)
-            : this(AttributeFactory.DEFAULT_ATTRIBUTE_FACTORY, reader, locale, iteratorType)
-        { }
+        protected SegmentingTokenizerBase(TextReader reader, BreakIterator iterator)
+            : this(AttributeFactory.DEFAULT_ATTRIBUTE_FACTORY, reader, iterator)
+        {
+        }
 
         /// <summary>
         /// Construct a new SegmenterBase, also supplying the AttributeFactory
         /// </summary>
-        protected SegmentingTokenizerBase(AttributeFactory factory, TextReader reader, BreakIterator.UBreakIteratorType iteratorType)
-            : this(factory, reader, LocaleUS, iteratorType) 
-        { }
-
-        protected SegmentingTokenizerBase(AttributeFactory factory, TextReader reader, Locale locale, BreakIterator.UBreakIteratorType iteratorType)
+        protected SegmentingTokenizerBase(AttributeFactory factory, TextReader reader, BreakIterator iterator)
             : base(factory, reader)
         {
             offsetAtt = AddAttribute<IOffsetAttribute>();
-            this.iteratorType = iteratorType;
-            this.locale = locale;
-            enumerator = Enumerable.Empty<Boundary>().GetEnumerator();
+            this.iterator = iterator;
         }
 
         public override sealed bool IncrementToken()
@@ -122,7 +102,7 @@ namespace Lucene.Net.Analysis.Util
         {
             base.Reset();
             wrapper.SetText(m_buffer, 0, 0);
-            enumerator = Enumerable.Empty<Boundary>().GetEnumerator();
+            iterator.SetText(new string(wrapper.Text, wrapper.Start, wrapper.Length));
             length = usableLength = m_offset = 0;
         }
 
@@ -193,16 +173,7 @@ namespace Lucene.Net.Analysis.Util
             }
 
             wrapper.SetText(m_buffer, 0, Math.Max(0, usableLength));
-
-            var text = new string(wrapper.Text, wrapper.Start, wrapper.Length);
-
-            if (enumerator != null)
-            {
-                enumerator.Dispose();
-                enumerator = null;
-            }
-
-            enumerator = BreakIterator.GetBoundaries(iteratorType, locale, text).ToList().GetEnumerator();
+            iterator.SetText(new string(wrapper.Text, wrapper.Start, wrapper.Length));
         }
 
         // TODO: refactor to a shared readFully somewhere
@@ -240,15 +211,22 @@ namespace Lucene.Net.Analysis.Util
 
             while (true)
             {
-                if (!enumerator.MoveNext())
+                int start = iterator.Current;
+
+                if (start == BreakIterator.DONE)
                 {
-                    return false;
+                    return false; // BreakIterator exhausted
                 }
 
-                var current = enumerator.Current;
+                // find the next set of boundaries
+                int end_Renamed = iterator.Next();
 
-                SetNextSentence(current.Start, current.End);
+                if (end_Renamed == BreakIterator.DONE)
+                {
+                    return false; // BreakIterator exhausted
+                }
 
+                SetNextSentence(start, end_Renamed);
                 if (IncrementWord())
                 {
                     return true;
@@ -256,23 +234,12 @@ namespace Lucene.Net.Analysis.Util
             }
         }
 
-        public override void Dispose()
-        {
-            if (enumerator != null)
-            {
-                enumerator.Dispose();
-                enumerator = null;
-            }
-
-            base.Dispose();
-        }
-
         /// <summary>
         /// Provides the next input sentence for analysis </summary>
-        protected internal abstract void SetNextSentence(int sentenceStart, int sentenceEnd);
+        protected abstract void SetNextSentence(int sentenceStart, int sentenceEnd);
 
         /// <summary>
         /// Returns true if another word is available </summary>
-        protected internal abstract bool IncrementWord();
+        protected abstract bool IncrementWord();
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/506f55a6/src/Lucene.Net.Tests.Analysis.Common/Analysis/Th/TestThaiAnalyzer.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Tests.Analysis.Common/Analysis/Th/TestThaiAnalyzer.cs b/src/Lucene.Net.Tests.Analysis.Common/Analysis/Th/TestThaiAnalyzer.cs
index 3654c56..5ff98e2 100644
--- a/src/Lucene.Net.Tests.Analysis.Common/Analysis/Th/TestThaiAnalyzer.cs
+++ b/src/Lucene.Net.Tests.Analysis.Common/Analysis/Th/TestThaiAnalyzer.cs
@@ -133,24 +133,15 @@ namespace Lucene.Net.Analysis.Th
         [Test]
         public virtual void TestRandomStrings()
         {
-            fail("This test is somehow crashing NUnit and causing it not to complete");
             CheckRandomData(Random(), new ThaiAnalyzer(TEST_VERSION_CURRENT), 1000 * RANDOM_MULTIPLIER);
         }
 
         /// <summary>
         /// blast some random large strings through the analyzer </summary>
         /// 
-#if NETSTANDARD
-        // NUnit does not have TimeoutAttribute for .NET Standard, so it can run for a long time.
-        // https://github.com/nunit/nunit/issues/1658
-        [LongRunningTest]
-#endif
         [Test]
         public virtual void TestRandomHugeStrings()
         {
-            // LUCENENET NOTE: This is NOT a long running test - it should only take a few seconds
-            fail("This test is somehow crashing NUnit and causing it not to complete");
-
             Random random = Random();
             CheckRandomData(random, new ThaiAnalyzer(TEST_VERSION_CURRENT), 100 * RANDOM_MULTIPLIER, 8192);
         }

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/506f55a6/src/Lucene.Net.Tests.Analysis.Common/Analysis/Util/TestSegmentingTokenizerBase.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Tests.Analysis.Common/Analysis/Util/TestSegmentingTokenizerBase.cs b/src/Lucene.Net.Tests.Analysis.Common/Analysis/Util/TestSegmentingTokenizerBase.cs
index d717edf..86fda8f 100644
--- a/src/Lucene.Net.Tests.Analysis.Common/Analysis/Util/TestSegmentingTokenizerBase.cs
+++ b/src/Lucene.Net.Tests.Analysis.Common/Analysis/Util/TestSegmentingTokenizerBase.cs
@@ -4,7 +4,8 @@ using Lucene.Net.Analysis.TokenAttributes;
 using Lucene.Net.Analysis.Util;
 using Lucene.Net.Support;
 using NUnit.Framework;
-\ufeffusing System;
+using System;
+using System.Globalization;
 using System.IO;
 using System.Text;
 using BreakIterator = Icu.BreakIterator;
@@ -130,20 +131,20 @@ namespace Lucene.Net.Analysis.Util
             internal IOffsetAttribute offsetAtt;
 
             public WholeSentenceTokenizer(TextReader reader)
-                : base(reader, new Locale("en-US"), BreakIterator.UBreakIteratorType.SENTENCE)
+                : base(reader, new IcuBreakIterator(BreakIterator.UBreakIteratorType.SENTENCE, CultureInfo.InvariantCulture)  /*new Locale("en-US"), BreakIterator.UBreakIteratorType.SENTENCE*/)
             {
                 termAtt = AddAttribute<ICharTermAttribute>();
                 offsetAtt = AddAttribute<IOffsetAttribute>();
             }
 
-            protected internal override void SetNextSentence(int sentenceStart, int sentenceEnd)
+            protected override void SetNextSentence(int sentenceStart, int sentenceEnd)
             {
                 this.sentenceStart = sentenceStart;
                 this.sentenceEnd = sentenceEnd;
                 hasSentence = true;
             }
 
-            protected internal override bool IncrementWord()
+            protected override bool IncrementWord()
             {
                 if (hasSentence)
                 {
@@ -175,14 +176,14 @@ namespace Lucene.Net.Analysis.Util
             internal IPositionIncrementAttribute posIncAtt;
 
             public SentenceAndWordTokenizer(TextReader reader)
-                : base(reader, new Locale("en-US"), BreakIterator.UBreakIteratorType.SENTENCE)
+                : base(reader, new IcuBreakIterator(BreakIterator.UBreakIteratorType.SENTENCE, CultureInfo.InvariantCulture) /*new Locale("en-US"), BreakIterator.UBreakIteratorType.SENTENCE*/)
             {
                 termAtt = AddAttribute<ICharTermAttribute>();
                 offsetAtt = AddAttribute<IOffsetAttribute>();
                 posIncAtt = AddAttribute<IPositionIncrementAttribute>();
             }
 
-            protected internal override void SetNextSentence(int sentenceStart, int sentenceEnd)
+            protected override void SetNextSentence(int sentenceStart, int sentenceEnd)
             {
                 this.wordStart = this.wordEnd = this.sentenceStart = sentenceStart;
                 this.sentenceEnd = sentenceEnd;
@@ -195,7 +196,7 @@ namespace Lucene.Net.Analysis.Util
                 posBoost = -1;
             }
 
-            protected internal override bool IncrementWord()
+            protected override bool IncrementWord()
             {
                 wordStart = wordEnd;
                 while (wordStart < sentenceEnd)