You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by mi...@apache.org on 2010/05/04 01:23:58 UTC

svn commit: r940671 [3/3] - in /lucene/dev/trunk: lucene/ lucene/src/java/org/apache/lucene/search/ lucene/src/java/org/apache/lucene/util/ lucene/src/java/org/apache/lucene/util/automaton/ lucene/src/test/org/apache/lucene/search/ lucene/src/test/org/...

Added: lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/automaton/UTF32ToUTF8.py
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/automaton/UTF32ToUTF8.py?rev=940671&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/automaton/UTF32ToUTF8.py (added)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/automaton/UTF32ToUTF8.py Mon May  3 23:23:57 2010
@@ -0,0 +1,351 @@
+import types
+import os
+import sys
+import random
+
+MAX_UNICODE = 0x10FFFF
+
+# TODO
+#   - could be more minimal
+#     - eg when bracket lands on a utf8 boundary, like 3 - 2047 -- they can share the two * edges
+#     - also 3 2048 or 3 65536 -- it should not have an * down the red path, but it does
+
+# MASKS[0] is bottom 1-bit
+# MASKS[1] is bottom 2-bits
+# ...
+
+utf8Ranges = [(0, 127),
+              (128, 2047),
+              (2048, 65535),
+              (65536, 1114111)]
+
+typeToColor = {'startend': 'purple',
+               'start': 'blue',
+               'end': 'red'}
+
+class FSA:
+
+  def __init__(self):
+    # maps fromNode -> (startUTF8, endUTF8, endNode)
+    self.states = {}
+    self.nodeUpto = 0
+
+  def run(self, bytes):
+    state = self.start
+    for b in bytes:
+      found = False
+      oldState = state
+      for label, s, e, n in self.states[state][1:]:
+        if b >= s and b <= e:
+          if found:
+            raise RuntimeError('state %s has ambiguous output for byte %s' % (oldState, b))
+          state = n
+          found = True
+      if not found:
+        return -1
+      
+    return state
+        
+  def addEdge(self, n1, n2, v1, v2, label):
+    """
+    Adds edge from n1-n2, utf8 byte range v1-v2.
+    """
+    assert n1 in self.states
+    assert type(v1) is types.IntType
+    assert type(v2) is types.IntType
+    self.states[n1].append((label, v1, v2, n2))
+
+  def addNode(self, label=None):
+    try:
+      self.states[self.nodeUpto] = [label]
+      return self.nodeUpto
+    finally:
+      self.nodeUpto += 1
+
+  def toDOT(self, label):
+    __l = []
+    w = __l.append
+    endNode = startNode = None
+    for id, details in self.states.items():
+      name = details[0]
+      if name == 'end':
+        endNode = id
+      elif name == 'start':
+        startNode = id
+
+    w('digraph %s {' % label)
+    w('  rankdir=LR;')
+    w('  size="8,5";')
+    w('  node [color=white label=""]; Ns;')
+
+    w('  node [color=black];')
+    w('  node [shape=doublecircle, label=""]; N%s [label="%s"];' % (endNode, endNode))
+    w('  node [shape=circle];')
+
+    w('  N%s [label="%s"];' % (startNode, startNode))
+    w('  Ns -> N%s;' % startNode)
+    for id, details in self.states.items():
+      edges = details[1:]
+      w('  N%s [label="%s"];' % (id, id))
+      for type, s, e, dest in edges:
+        c = typeToColor.get(type, 'black')
+        if type == 'all*':
+          # special case -- matches any utf8 byte at this point
+          label = '*'
+        elif s == e:
+          label = '%s' % binary(s)
+        else:
+          label = '%s-%s' % (binary(s), binary(e))
+        w('  N%s -> N%s [label="%s" color="%s"];' % (id, dest, label, c))
+      if name == 'end':
+        endNode = id
+      elif name == 'start':
+        startNode = id
+    w('}')
+    return '\n'.join(__l)
+
+  def toPNG(self, label, pngOut):
+    open('tmp.dot', 'wb').write(self.toDOT(label))
+    if os.system('dot -Tpng tmp.dot -o %s' % pngOut):
+      raise RuntimeException('dot failed')
+    
+
+MASKS = []
+v = 2
+for i in range(32):
+  MASKS.append(v-1)
+  v *= 2
+
+def binary(x):
+  if x == 0:
+    return '00000000'
+  
+  l = []
+  while x > 0:
+    if x & 1 == 1:
+      l.append('1')
+    else:
+      l.append('0')
+    x = x >> 1
+
+  # big endian!
+  l.reverse()
+
+  l2 = []
+  while len(l) > 0:
+    s = ''.join(l[-8:])
+    if len(s) < 8:
+      s = '0'*(8-len(s)) + s
+    l2.append(s)
+    del l[-8:]
+
+  return ' '.join(l2)
+
+def getUTF8Rest(code, numBytes):
+  l = []
+  for i in range(numBytes):
+    l.append((128 | (code & MASKS[5]), 6))
+    code = code >> 6
+  l.reverse()
+  return tuple(l)
+
+def toUTF8(code):
+  # code = Unicode code point
+  assert code >= 0
+  assert code <= MAX_UNICODE
+
+  if code < 128:
+    # 0xxxxxxx
+    bytes = ((code, 7),)
+  elif code < 2048:
+    # 110yyyxx 10xxxxxx
+    byte1 = (6 << 5) | (code >> 6)
+    bytes = ((byte1, 5),) + getUTF8Rest(code, 1)
+  elif code < 65536:
+    # 1110yyyy 10yyyyxx 10xxxxxx
+    len = 3
+    byte1 = (14 << 4) | (code >> 12)
+    bytes = ((byte1, 4),) + getUTF8Rest(code, 2)
+  else:
+    # 11110zzz 10zzyyyy 10yyyyxx 10xxxxxx
+    len = 4
+    byte1 = (30 << 3) | (code >> 18)
+    bytes = ((byte1, 3),) + getUTF8Rest(code, 3)
+
+  return bytes
+
+def all(fsa, startNode, endNode, startCode, endCode, left):
+  if len(left) == 0:
+    fsa.addEdge(startNode, endNode, startCode, endCode, 'all')
+  else:
+    lastN = fsa.addNode()
+    fsa.addEdge(startNode, lastN, startCode, endCode, 'all')
+    while len(left) > 1:
+      n = fsa.addNode()
+      fsa.addEdge(lastN, n, 128, 191, 'all*')
+      left = left[1:]
+      lastN = n
+    fsa.addEdge(lastN, endNode, 128, 191, 'all*')
+          
+def start(fsa, startNode, endNode, utf8, doAll):
+  if len(utf8) == 1:
+    fsa.addEdge(startNode, endNode, utf8[0][0], utf8[0][0] | MASKS[utf8[0][1]-1], 'start')
+  else:
+    n = fsa.addNode()
+    fsa.addEdge(startNode, n, utf8[0][0], utf8[0][0], 'start')
+    start(fsa, n, endNode, utf8[1:], True)
+    end = utf8[0][0] | MASKS[utf8[0][1]-1]
+    if doAll and utf8[0][0] != end:
+      all(fsa, startNode, endNode, utf8[0][0]+1, end, utf8[1:])
+
+def end(fsa, startNode, endNode, utf8, doAll):
+  if len(utf8) == 1:
+    fsa.addEdge(startNode, endNode, utf8[0][0] & ~MASKS[utf8[0][1]-1], utf8[0][0], 'end')
+  else:
+    if utf8[0][1] == 5:
+      # special case -- avoid created unused edges (utf8 doesn't accept certain byte sequences):
+      start = 194
+    else:
+      start = utf8[0][0] & (~MASKS[utf8[0][1]-1])
+    if doAll and utf8[0][0] != start:
+      all(fsa, startNode, endNode, start, utf8[0][0]-1, utf8[1:])
+    n = fsa.addNode()
+    fsa.addEdge(startNode, n, utf8[0][0], utf8[0][0], 'end')
+    end(fsa, n, endNode, utf8[1:], True)
+
+def build(fsa,
+          startNode, endNode,
+          startUTF8, endUTF8):
+
+  # Break into start, middle, end:
+  if startUTF8[0][0] == endUTF8[0][0]:
+    # Degen case: lead with the same byte:
+    if len(startUTF8) == 1 and len(endUTF8) == 1:
+      fsa.addEdge(startNode, endNode, startUTF8[0][0], endUTF8[0][0], 'startend')
+      return
+    else:
+      assert len(startUTF8) != 1
+      assert len(endUTF8) != 1
+      n = fsa.addNode()
+      # single value edge
+      fsa.addEdge(startNode, n, startUTF8[0][0], startUTF8[0][0], 'single')
+      build(fsa, n, endNode, startUTF8[1:], endUTF8[1:])
+  elif len(startUTF8) == len(endUTF8):
+    if len(startUTF8) == 1:
+      fsa.addEdge(startNode, endNode, startUTF8[0][0], endUTF8[0][0], 'startend')
+    else:
+      start(fsa, startNode, endNode, startUTF8, False)
+      if endUTF8[0][0] - startUTF8[0][0] > 1:
+        all(fsa, startNode, endNode, startUTF8[0][0]+1, endUTF8[0][0]-1, startUTF8[1:])
+      end(fsa, startNode, endNode, endUTF8, False)
+  else:
+    # start
+    start(fsa, startNode, endNode, startUTF8, True)
+
+    # possibly middle
+    byteCount = 1+len(startUTF8)
+    while byteCount < len(endUTF8):
+      s = toUTF8(utf8Ranges[byteCount-1][0])
+      e = toUTF8(utf8Ranges[byteCount-1][1])
+      all(fsa, startNode, endNode,
+          s[0][0],
+          e[0][0],
+          s[1:])
+      byteCount += 1
+
+    # end
+    end(fsa, startNode, endNode, endUTF8, True)
+
+def main():
+
+  if len(sys.argv) not in (3, 4):
+    print
+    print 'Usage: python %s startUTF32 endUTF32 [testCode]' % sys.argv[0]
+    print
+    sys.exit(1)
+
+  utf32Start = int(sys.argv[1])
+  utf32End = int(sys.argv[2])
+
+  if utf32Start > utf32End:
+    print 'ERROR: start must be <= end'
+    sys.exit(1)
+
+  fsa = FSA()
+  fsa.start = fsa.addNode('start')
+  fsa.end = fsa.addNode('end')
+
+  print 's=%s' % ' '.join([binary(x[0]) for x in toUTF8(utf32Start)])
+  print 'e=%s' % ' '.join([binary(x[0]) for x in toUTF8(utf32End)])
+
+  if len(sys.argv) == 4:
+    print 't=%s [%s]' % \
+          (' '.join([binary(x[0]) for x in toUTF8(int(sys.argv[3]))]),
+           ' '.join(['%2x' % x[0] for x in toUTF8(int(sys.argv[3]))]))
+  
+  build(fsa, fsa.start, fsa.end,
+        toUTF8(utf32Start),
+        toUTF8(utf32End))
+
+  fsa.toPNG('test', '/tmp/outpy.png')
+  print 'Saved to /tmp/outpy.png...'
+
+  test(fsa, utf32Start, utf32End, 100000);
+
+def test(fsa, utf32Start, utf32End, count):
+
+  # verify correct ints are accepted
+  for i in range(count):
+    r = random.randint(utf32Start, utf32End)
+    dest = fsa.run([tup[0] for tup in toUTF8(r)])
+    if dest != fsa.end:
+      print 'FAILED: valid %s (%s) is not accepted' % (r, ' '.join([binary(x[0]) for x in toUTF8(r)]))
+      return False
+
+  invalidRange = MAX_UNICODE - (utf32End - utf32Start + 1)
+  if invalidRange >= 0:
+    # verify invalid ints are not accepted
+    for i in range(count):
+      r = random.randint(0, invalidRange-1)
+      if r >= utf32Start:
+        r = utf32End + 1 + r - utf32Start
+      dest = fsa.run([tup[0] for tup in toUTF8(r)])
+      if dest != -1:
+        print 'FAILED: invalid %s (%s) is accepted' % (r, ' '.join([binary(x[0]) for x in toUTF8(r)]))
+        return False
+
+  return True
+
+def stress():
+
+  print 'Testing...'
+
+  iter = 0
+  while True:
+    if iter % 10 == 0:
+      print '%s...' % iter
+    iter += 1
+
+    v1 = random.randint(0, MAX_UNICODE)
+    v2 = random.randint(0, MAX_UNICODE)
+    if v2 < v1:
+      v1, v2 = v2, v1
+
+    utf32Start = v1
+    utf32End = v2
+
+    fsa = FSA()
+    fsa.start = fsa.addNode('start')
+    fsa.end = fsa.addNode('end')
+    build(fsa, fsa.start, fsa.end,
+          toUTF8(utf32Start),
+          toUTF8(utf32End))
+
+    if not test(fsa, utf32Start, utf32End, 10000):
+      print 'FAILED on utf32Start=%s utf32End=%s' % (utf32Start, utf32End)
+
+if __name__ == '__main__':
+  if len(sys.argv) > 1:
+    main()
+  else:
+    stress()

Propchange: lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/automaton/UTF32ToUTF8.py
------------------------------------------------------------------------------
    svn:eol-style = native

Modified: lucene/dev/trunk/lucene/src/test/org/apache/lucene/search/TestAutomatonQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/test/org/apache/lucene/search/TestAutomatonQuery.java?rev=940671&r1=940670&r2=940671&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/test/org/apache/lucene/search/TestAutomatonQuery.java (original)
+++ lucene/dev/trunk/lucene/src/test/org/apache/lucene/search/TestAutomatonQuery.java Mon May  3 23:23:57 2010
@@ -145,10 +145,8 @@ public class TestAutomatonQuery extends 
         .makeString("foobar"));
     
     assertEquals(a1, a2);
-    assertEquals(a1.hashCode(), a2.hashCode());
     
     assertEquals(a1, a3);
-    assertEquals(a1.hashCode(), a3.hashCode());
     
     assertEquals(a1.toString(), a3.toString());
     

Modified: lucene/dev/trunk/lucene/src/test/org/apache/lucene/search/TestAutomatonQueryUnicode.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/test/org/apache/lucene/search/TestAutomatonQueryUnicode.java?rev=940671&r1=940670&r2=940671&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/test/org/apache/lucene/search/TestAutomatonQueryUnicode.java (original)
+++ lucene/dev/trunk/lucene/src/test/org/apache/lucene/search/TestAutomatonQueryUnicode.java Mon May  3 23:23:57 2010
@@ -124,55 +124,4 @@ public class TestAutomatonQueryUnicode e
     Automaton a = new RegExp("((\uD866\uDF05)|\uFB94).*").toAutomaton();
     assertAutomatonHits(2, a);
   }
-  
-  /**
-   * Test that AutomatonQuery properly seeks to supplementary characters.
-   * Transitions are modeled as UTF-16 code units, so without special handling
-   * by default it will try to seek to a lead surrogate with some DFAs
-   */
-  public void testSeekSurrogate() throws IOException {
-    Automaton a = new RegExp("\uD866[a\uDF05\uFB93][a-z]{0,5}[fl]").toAutomaton();
-    assertAutomatonHits(1, a);
-  }
-  
-  /**
-   * Try seeking to an ending lead surrogate.
-   */
-  public void testSeekSurrogate2() throws IOException {
-    Automaton a = new RegExp("\uD866(\uDF06ghijkl)?").toAutomaton();
-    assertAutomatonHits(1, a);
-  }
-  
-  /**
-   * Try seeking to an starting trail surrogate.
-   */
-  public void testSeekSurrogate3() throws IOException {
-    Automaton a = new RegExp("[\uDF06\uFB94]mnopqr").toAutomaton();
-    assertAutomatonHits(1, a);
-  }
-  
-  /**
-   * Try seeking to an medial/final trail surrogate.
-   */
-  public void testSeekSurrogate4() throws IOException {
-    Automaton a = new RegExp("a[\uDF06\uFB94]bc").toAutomaton();
-    assertAutomatonHits(1, a);
-  }
-  
-  /**
-   * Ensure the 'constant suffix' does not contain a leading trail surrogate.
-   */
-  public void testSurrogateSuffix() throws IOException {
-    Automaton a = new RegExp(".*[\uD865\uD866]\uDF06ghijkl").toAutomaton();
-    assertAutomatonHits(1, a);
-  }
-  
-  /**
-   * Try when the constant suffix is only a leading trail surrogate.
-   * instead this must use an empty suffix.
-   */
-  public void testSurrogateSuffix2() throws IOException {
-    Automaton a = new RegExp(".*\uDF05").toAutomaton();
-    assertAutomatonHits(1, a);
-  }
 }

Modified: lucene/dev/trunk/lucene/src/test/org/apache/lucene/search/TestFuzzyQuery2.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/test/org/apache/lucene/search/TestFuzzyQuery2.java?rev=940671&r1=940670&r2=940671&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/test/org/apache/lucene/search/TestFuzzyQuery2.java (original)
+++ lucene/dev/trunk/lucene/src/test/org/apache/lucene/search/TestFuzzyQuery2.java Mon May  3 23:23:57 2010
@@ -56,6 +56,20 @@ public class TestFuzzyQuery2 extends Luc
   static final float epsilon = 0.00001f;
 
   public void testFromTestData() throws Exception {
+    // TODO: randomize!
+    assertFromTestData(new int[] { 0x40, 0x41 });
+    assertFromTestData(new int[] { 0x40, 0x0195 });
+    assertFromTestData(new int[] { 0x40, 0x0906 });
+    assertFromTestData(new int[] { 0x40, 0x1040F });
+    assertFromTestData(new int[] { 0x0194, 0x0195 });
+    assertFromTestData(new int[] { 0x0194, 0x0906 });
+    assertFromTestData(new int[] { 0x0194, 0x1040F });
+    assertFromTestData(new int[] { 0x0905, 0x0906 });
+    assertFromTestData(new int[] { 0x0905, 0x1040F });
+    assertFromTestData(new int[] { 0x1040E, 0x1040F });
+  }
+
+  public void assertFromTestData(int codePointTable[]) throws Exception {
     InputStream stream = getClass().getResourceAsStream("fuzzyTestData.txt");
     BufferedReader reader = new BufferedReader(new InputStreamReader(stream, "UTF-8"));
     
@@ -71,7 +85,7 @@ public class TestFuzzyQuery2 extends Luc
     doc.add(field);
     
     for (int i = 0; i < terms; i++) {
-      field.setValue(Integer.toBinaryString(i));
+      field.setValue(mapInt(codePointTable, i));
       writer.addDocument(doc);
     }
     
@@ -82,7 +96,7 @@ public class TestFuzzyQuery2 extends Luc
     String line;
     while ((line = reader.readLine()) != null) {
       String params[] = line.split(",");
-      String query = Integer.toBinaryString(Integer.parseInt(params[0]));
+      String query = mapInt(codePointTable, Integer.parseInt(params[0]));
       int prefix = Integer.parseInt(params[1]);
       int pqSize = Integer.parseInt(params[2]);
       float minScore = Float.parseFloat(params[3]);
@@ -101,6 +115,15 @@ public class TestFuzzyQuery2 extends Luc
     dir.close();
   }
   
+  /* map bits to unicode codepoints */
+  private static String mapInt(int codePointTable[], int i) {
+    StringBuilder sb = new StringBuilder();
+    String binary = Integer.toBinaryString(i);
+    for (int j = 0; j < binary.length(); j++)
+      sb.appendCodePoint(codePointTable[binary.charAt(j) - '0']);
+    return sb.toString();
+  }
+
   /* Code to generate test data
   public static void main(String args[]) throws Exception {
     int bits = 3;

Modified: lucene/dev/trunk/lucene/src/test/org/apache/lucene/search/TestRegexpRandom2.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/test/org/apache/lucene/search/TestRegexpRandom2.java?rev=940671&r1=940670&r2=940671&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/test/org/apache/lucene/search/TestRegexpRandom2.java (original)
+++ lucene/dev/trunk/lucene/src/test/org/apache/lucene/search/TestRegexpRandom2.java Mon May  3 23:23:57 2010
@@ -31,9 +31,11 @@ import org.apache.lucene.store.RAMDirect
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.util.UnicodeUtil;
+import org.apache.lucene.util._TestUtil;
 import org.apache.lucene.util.automaton.Automaton;
+import org.apache.lucene.util.automaton.AutomatonTestUtil;
+import org.apache.lucene.util.automaton.CharacterRunAutomaton;
 import org.apache.lucene.util.automaton.RegExp;
-import org.apache.lucene.util.automaton.RunAutomaton;
 
 /**
  * Create an index with random unicode terms
@@ -46,17 +48,17 @@ public class TestRegexpRandom2 extends L
   @Override
   protected void setUp() throws Exception {
     super.setUp();
-    random = newRandom(System.nanoTime());
+    random = newRandom();
     RAMDirectory dir = new RAMDirectory();
     IndexWriter writer = new IndexWriter(dir, new KeywordAnalyzer(),
         IndexWriter.MaxFieldLength.UNLIMITED);
     
     Document doc = new Document();
-    Field field = new Field("field", "", Field.Store.YES, Field.Index.ANALYZED);
+    Field field = new Field("field", "", Field.Store.NO, Field.Index.ANALYZED);
     doc.add(field);
     
-    for (int i = 0; i < 1000; i++) {
-      field.setValue(randomString());
+    for (int i = 0; i < 2000; i++) {
+      field.setValue(_TestUtil.randomUnicodeString(random));
       writer.addDocument(doc);
     }
     
@@ -87,7 +89,7 @@ public class TestRegexpRandom2 extends L
     }
 
     private class SimpleAutomatonTermsEnum extends FilteredTermsEnum {
-      RunAutomaton runAutomaton = new RunAutomaton(automaton);
+      CharacterRunAutomaton runAutomaton = new CharacterRunAutomaton(automaton);
       UnicodeUtil.UTF16Result utf16 = new UnicodeUtil.UTF16Result();
 
       private SimpleAutomatonTermsEnum(IndexReader reader, String field) throws IOException {
@@ -111,25 +113,14 @@ public class TestRegexpRandom2 extends L
   
   /** test a bunch of random regular expressions */
   public void testRegexps() throws Exception {
-      for (int i = 0; i < 500; i++)
-        assertSame(randomRegex());
+      for (int i = 0; i < 1000; i++)
+        assertSame(AutomatonTestUtil.randomRegexp(random).toString());
   }
   
   /** check that the # of hits is the same as from a very
    * simple regexpquery implementation.
    */
-  private void assertSame(String regexp) throws IOException {
-    // we will generate some illegal syntax regular expressions...
-    try {
-      new RegExp(regexp).toAutomaton();
-    } catch (Exception e) {
-      return;
-    }
-    
-    // we will also generate some undefined unicode queries
-    if (!UnicodeUtil.validUTF16String(regexp))
-      return;
-    
+  private void assertSame(String regexp) throws IOException {   
     RegexpQuery smart = new RegexpQuery(new Term("field", regexp));
     DumbRegexpQuery dumb = new DumbRegexpQuery(new Term("field", regexp));
     
@@ -143,79 +134,7 @@ public class TestRegexpRandom2 extends L
     
     TopDocs smartDocs = searcher.search(smart, 25);
     TopDocs dumbDocs = searcher.search(dumb, 25);
-
-    assertEquals(dumbDocs.totalHits, smartDocs.totalHits);
-  }
-  
-  char buffer[] = new char[20];
-  
-  // start is inclusive and end is exclusive
-  public int nextInt(int start, int end) {
-    return start + random.nextInt(end - start);
-  }
-  
-  public String randomString() {
-    final int end = random.nextInt(20);
-    if (buffer.length < 1 + end) {
-      char[] newBuffer = new char[(int) ((1 + end) * 1.25)];
-      System.arraycopy(buffer, 0, newBuffer, 0, buffer.length);
-      buffer = newBuffer;
-    }
-    for (int i = 0; i < end - 1; i++) {
-      int t = random.nextInt(6);
-      if (0 == t && i < end - 1) {
-        // Make a surrogate pair
-        // High surrogate
-        buffer[i++] = (char) nextInt(0xd800, 0xdc00);
-        // Low surrogate
-        buffer[i] = (char) nextInt(0xdc00, 0xe000);
-      } else if (t <= 1) buffer[i] = (char) random.nextInt(0x80);
-      else if (2 == t) buffer[i] = (char) nextInt(0x80, 0x800);
-      else if (3 == t) buffer[i] = (char) nextInt(0x800, 0xd800);
-      else if (4 == t) buffer[i] = (char) nextInt(0xe000, 0xffff);
-      else if (5 == t) {
-        // Illegal unpaired surrogate
-        if (random.nextBoolean()) buffer[i] = (char) nextInt(0xd800, 0xdc00);
-        else buffer[i] = (char) nextInt(0xdc00, 0xe000);
-      }
-    }
-    return new String(buffer, 0, end);
-  }
-  
-  // a random string biased towards populating a ton of operators
-  public String randomRegex() {
-    final int end = random.nextInt(20);
-    if (buffer.length < 1 + end) {
-      char[] newBuffer = new char[(int) ((1 + end) * 1.25)];
-      System.arraycopy(buffer, 0, newBuffer, 0, buffer.length);
-      buffer = newBuffer;
-    }
-    for (int i = 0; i < end - 1; i++) {
-      int t = random.nextInt(10);
-      if (0 == t && i < end - 1) {
-        // Make a surrogate pair
-        // High surrogate
-        buffer[i++] = (char) nextInt(0xd800, 0xdc00);
-        // Low surrogate
-        buffer[i] = (char) nextInt(0xdc00, 0xe000);
-      } else if (t <= 1) buffer[i] = (char) random.nextInt(0x80);
-      else if (2 == t) buffer[i] = (char) nextInt(0x80, 0x800);
-      else if (3 == t) buffer[i] = (char) nextInt(0x800, 0xd800);
-      else if (4 == t) buffer[i] = (char) nextInt(0xe000, 0xffff);
-      else if (5 == t) {
-        // Illegal unpaired surrogate
-        if (random.nextBoolean()) buffer[i] = (char) nextInt(0xd800, 0xdc00);
-        else buffer[i] = (char) nextInt(0xdc00, 0xe000);
-      } else if (6 == t) {
-        buffer[i] = '.';
-      } else if (7 == t) {
-        buffer[i] = '?';
-      } else if (8 == t) {
-        buffer[i] = '*';
-      } else if (9 == t) {
-        buffer[i] = '+';
-      }
-    }
-    return new String(buffer, 0, end);
+    
+    assertEquals("for re:" + regexp, dumbDocs.totalHits, smartDocs.totalHits);
   }
 }

Modified: lucene/dev/trunk/lucene/src/test/org/apache/lucene/util/TestArrayUtil.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/test/org/apache/lucene/util/TestArrayUtil.java?rev=940671&r1=940670&r2=940671&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/test/org/apache/lucene/util/TestArrayUtil.java (original)
+++ lucene/dev/trunk/lucene/src/test/org/apache/lucene/util/TestArrayUtil.java Mon May  3 23:23:57 2010
@@ -102,4 +102,17 @@ public class TestArrayUtil extends Lucen
 
   }
 
+  public void testSliceEquals() {
+    String left = "this is equal";
+    String right = left;
+    char[] leftChars = left.toCharArray();
+    char[] rightChars = right.toCharArray();
+    assertTrue(left + " does not equal: " + right, ArrayUtil.equals(leftChars, 0, rightChars, 0, left.length()));
+    
+    assertFalse(left + " does not equal: " + right, ArrayUtil.equals(leftChars, 1, rightChars, 0, left.length()));
+    assertFalse(left + " does not equal: " + right, ArrayUtil.equals(leftChars, 1, rightChars, 2, left.length()));
+
+    assertFalse(left + " does not equal: " + right, ArrayUtil.equals(leftChars, 25, rightChars, 0, left.length()));
+    assertFalse(left + " does not equal: " + right, ArrayUtil.equals(leftChars, 12, rightChars, 0, left.length()));
+  }
 }

Modified: lucene/dev/trunk/lucene/src/test/org/apache/lucene/util/TestUnicodeUtil.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/test/org/apache/lucene/util/TestUnicodeUtil.java?rev=940671&r1=940670&r2=940671&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/test/org/apache/lucene/util/TestUnicodeUtil.java (original)
+++ lucene/dev/trunk/lucene/src/test/org/apache/lucene/util/TestUnicodeUtil.java Mon May  3 23:23:57 2010
@@ -17,6 +17,8 @@ package org.apache.lucene.util;
  * limitations under the License.
  */
 
+import java.util.Random;
+
 
 /*
  * Some of this code came from the excellent Unicode
@@ -81,4 +83,47 @@ public class TestUnicodeUtil extends Luc
     assertEquals("dogs\uE000", UnicodeUtil.nextValidUTF16String("dogs\uDC00"));
     assertEquals("\uE000", UnicodeUtil.nextValidUTF16String("\uDC00dogs"));
   }
+
+  public void testCodePointCount() {
+    final Random r = newRandom();
+    BytesRef utf8 = new BytesRef(20);
+    for(int i=0;i<50000;i++) {
+      final String s = _TestUtil.randomUnicodeString(r);
+      UnicodeUtil.UTF16toUTF8(s, 0, s.length(), utf8);
+      assertEquals(s.codePointCount(0, s.length()),
+                   UnicodeUtil.codePointCount(utf8));
+    }
+  }
+
+  public void testUTF8toUTF32() {
+    final Random r = newRandom();
+    BytesRef utf8 = new BytesRef(20);
+    IntsRef utf32 = new IntsRef(20);
+    int[] codePoints = new int[20];
+    for(int i=0;i<50000;i++) {
+      final String s = _TestUtil.randomUnicodeString(r);
+      UnicodeUtil.UTF16toUTF8(s, 0, s.length(), utf8);
+      UnicodeUtil.UTF8toUTF32(utf8, utf32);
+      
+      int charUpto = 0;
+      int intUpto = 0;
+      while(charUpto < s.length()) {
+        final int cp = s.codePointAt(charUpto);
+        codePoints[intUpto++] = cp;
+        charUpto += Character.charCount(cp);
+      }
+      if (!ArrayUtil.equals(codePoints, 0, utf32.ints, utf32.offset, intUpto)) {
+        System.out.println("FAILED");
+        for(int j=0;j<s.length();j++) {
+          System.out.println("  char[" + j + "]=" + Integer.toHexString(s.charAt(j)));
+        }
+        System.out.println();
+        assertEquals(intUpto, utf32.length);
+        for(int j=0;j<intUpto;j++) {
+          System.out.println("  " + Integer.toHexString(utf32.ints[j]) + " vs " + Integer.toHexString(codePoints[j]));
+        }
+        fail("mismatch");
+      }
+    }
+  }
 }

Modified: lucene/dev/trunk/lucene/src/test/org/apache/lucene/util/_TestUtil.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/test/org/apache/lucene/util/_TestUtil.java?rev=940671&r1=940670&r2=940671&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/test/org/apache/lucene/util/_TestUtil.java (original)
+++ lucene/dev/trunk/lucene/src/test/org/apache/lucene/util/_TestUtil.java Mon May  3 23:23:57 2010
@@ -111,4 +111,33 @@ public class _TestUtil {
     buf.append("]");
     return buf.toString();
   }
+  /** start and end are BOTH inclusive */
+  public static int nextInt(Random r, int start, int end) {
+    return start + r.nextInt(end-start+1);
+  }
+
+  /** Returns random string, including full unicode range. */
+  public static String randomUnicodeString(Random r) {
+    final int end = r.nextInt(20);
+    if (end == 0) {
+      // allow 0 length
+      return "";
+    }
+    final char[] buffer = new char[end];
+    for (int i = 0; i < end; i++) {
+      int t = r.nextInt(5);
+      if (0 == t && i < end - 1) {
+        // Make a surrogate pair
+        // High surrogate
+        buffer[i++] = (char) nextInt(r, 0xd800, 0xdbff);
+        // Low surrogate
+        buffer[i] = (char) nextInt(r, 0xdc00, 0xdfff);
+      }
+      else if (t <= 1) buffer[i] = (char) r.nextInt(0x80);
+      else if (2 == t) buffer[i] = (char) nextInt(r, 0x80, 0x800);
+      else if (3 == t) buffer[i] = (char) nextInt(r, 0x800, 0xd7ff);
+      else if (4 == t) buffer[i] = (char) nextInt(r, 0xe000, 0xffff);
+    }
+    return new String(buffer, 0, end);
+  }
 }

Added: lucene/dev/trunk/lucene/src/test/org/apache/lucene/util/automaton/AutomatonTestUtil.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/test/org/apache/lucene/util/automaton/AutomatonTestUtil.java?rev=940671&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/src/test/org/apache/lucene/util/automaton/AutomatonTestUtil.java (added)
+++ lucene/dev/trunk/lucene/src/test/org/apache/lucene/util/automaton/AutomatonTestUtil.java Mon May  3 23:23:57 2010
@@ -0,0 +1,68 @@
+package org.apache.lucene.util.automaton;
+
+/**
+ * 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.
+ */
+
+import java.util.Random;
+
+import org.apache.lucene.util.UnicodeUtil;
+import org.apache.lucene.util._TestUtil;
+
+public class AutomatonTestUtil {
+  /** Returns random string, including full unicode range. */
+  public static RegExp randomRegexp(Random r) {
+    while (true) {
+      String regexp = randomRegexpString(r);
+      // we will also generate some undefined unicode queries
+      if (!UnicodeUtil.validUTF16String(regexp))
+        continue;
+      try {
+        return new RegExp(regexp, RegExp.NONE);
+      } catch (Exception e) {}
+    }
+  }
+
+  private static String randomRegexpString(Random r) {
+    final int end = r.nextInt(20);
+    if (end == 0) {
+      // allow 0 length
+      return "";
+    }
+    final char[] buffer = new char[end];
+    for (int i = 0; i < end; i++) {
+      int t = r.nextInt(11);
+      if (0 == t && i < end - 1) {
+        // Make a surrogate pair
+        // High surrogate
+        buffer[i++] = (char) _TestUtil.nextInt(r, 0xd800, 0xdbff);
+        // Low surrogate
+        buffer[i] = (char) _TestUtil.nextInt(r, 0xdc00, 0xdfff);
+      }
+      else if (t <= 1) buffer[i] = (char) r.nextInt(0x80);
+      else if (2 == t) buffer[i] = (char) _TestUtil.nextInt(r, 0x80, 0x800);
+      else if (3 == t) buffer[i] = (char) _TestUtil.nextInt(r, 0x800, 0xd7ff);
+      else if (4 == t) buffer[i] = (char) _TestUtil.nextInt(r, 0xe000, 0xffff);
+      else if (5 == t) buffer[i] = '.';
+      else if (6 == t) buffer[i] = '?';
+      else if (7 == t) buffer[i] = '*';
+      else if (8 == t) buffer[i] = '+';
+      else if (9 == t) buffer[i] = '(';
+      else if (10 == t) buffer[i] = ')';
+    }
+    return new String(buffer, 0, end);
+  }
+}

Propchange: lucene/dev/trunk/lucene/src/test/org/apache/lucene/util/automaton/AutomatonTestUtil.java
------------------------------------------------------------------------------
    svn:eol-style = native

Modified: lucene/dev/trunk/lucene/src/test/org/apache/lucene/util/automaton/TestBasicOperations.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/test/org/apache/lucene/util/automaton/TestBasicOperations.java?rev=940671&r1=940670&r2=940671&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/test/org/apache/lucene/util/automaton/TestBasicOperations.java (original)
+++ lucene/dev/trunk/lucene/src/test/org/apache/lucene/util/automaton/TestBasicOperations.java Mon May  3 23:23:57 2010
@@ -16,8 +16,11 @@ package org.apache.lucene.util.automaton
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+
 import org.apache.lucene.util.LuceneTestCase;
 
+import java.util.Random;
+
 public class TestBasicOperations extends LuceneTestCase { 
   /** Test optimization to concatenate() */
   public void testSingletonConcatenate() {
@@ -26,7 +29,7 @@ public class TestBasicOperations extends
     Automaton other = BasicAutomata.makeCharRange('5', '7');
     Automaton concat = BasicOperations.concatenate(singleton, other);
     assertTrue(concat.isDeterministic());
-    assertEquals(BasicOperations.concatenate(expandedSingleton, other), concat);
+    assertTrue(BasicOperations.sameLanguage(BasicOperations.concatenate(expandedSingleton, other), concat));
   }
   
   /** Test optimization to concatenate() to an NFA */
@@ -38,7 +41,7 @@ public class TestBasicOperations extends
         BasicAutomata.makeString("three"));
     Automaton concat = BasicOperations.concatenate(singleton, nfa);
     assertFalse(concat.isDeterministic());
-    assertEquals(BasicOperations.concatenate(expandedSingleton, nfa), concat);
+    assertTrue(BasicOperations.sameLanguage(BasicOperations.concatenate(expandedSingleton, nfa), concat));
   }
   
   /** Test optimization to concatenate() with empty String */
@@ -49,9 +52,9 @@ public class TestBasicOperations extends
     Automaton concat1 = BasicOperations.concatenate(expandedSingleton, other);
     Automaton concat2 = BasicOperations.concatenate(singleton, other);
     assertTrue(concat2.isDeterministic());
-    assertEquals(concat1, concat2);
-    assertEquals(other, concat1);
-    assertEquals(other, concat2);
+    assertTrue(BasicOperations.sameLanguage(concat1, concat2));
+    assertTrue(BasicOperations.sameLanguage(other, concat1));
+    assertTrue(BasicOperations.sameLanguage(other, concat2));
   }
   
   /** Test optimization to concatenate() with empty String to an NFA */
@@ -64,8 +67,50 @@ public class TestBasicOperations extends
     Automaton concat1 = BasicOperations.concatenate(expandedSingleton, nfa);
     Automaton concat2 = BasicOperations.concatenate(singleton, nfa);
     assertFalse(concat2.isDeterministic());
-    assertEquals(concat1, concat2);
-    assertEquals(nfa, concat1);
-    assertEquals(nfa, concat2);
+    assertTrue(BasicOperations.sameLanguage(concat1, concat2));
+    assertTrue(BasicOperations.sameLanguage(nfa, concat1));
+    assertTrue(BasicOperations.sameLanguage(nfa, concat2));
+  }
+
+  /** Test singletons work correctly */
+  public void testSingleton() {
+    Automaton singleton = BasicAutomata.makeString("foobar");
+    Automaton expandedSingleton = singleton.cloneExpanded();
+    assertTrue(BasicOperations.sameLanguage(singleton, expandedSingleton));
+    
+    singleton = BasicAutomata.makeString("\ud801\udc1c");
+    expandedSingleton = singleton.cloneExpanded();
+    //assertEquals(singleton, expandedSingleton);
+  }
+
+  public void testGetRandomAcceptedString() throws Throwable {
+    final Random r = newRandom();
+    final int ITER1 = 100;
+    final int ITER2 = 100;
+    for(int i=0;i<ITER1;i++) {
+
+      final RegExp re = AutomatonTestUtil.randomRegexp(r);
+      final Automaton a = re.toAutomaton();
+      assertFalse(BasicOperations.isEmpty(a));
+
+      final BasicOperations.RandomAcceptedStrings rx = new BasicOperations.RandomAcceptedStrings(a);
+      for(int j=0;j<ITER2;j++) {
+        int[] acc = null;
+        try {
+          acc = rx.getRandomAcceptedString(r);
+          final String s = new String(acc, 0, acc.length);
+          assertTrue(BasicOperations.run(a, s));
+        } catch (Throwable t) {
+          System.out.println("regexp: " + re);
+          if (acc != null) {
+            System.out.println("fail acc re=" + re + " count=" + acc.length);
+            for(int k=0;k<acc.length;k++) {
+              System.out.println("  " + Integer.toHexString(acc[k]));
+            }
+          }
+          throw t;
+        }
+      }
+    }
   }
 }

Added: lucene/dev/trunk/lucene/src/test/org/apache/lucene/util/automaton/TestDeterminism.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/test/org/apache/lucene/util/automaton/TestDeterminism.java?rev=940671&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/src/test/org/apache/lucene/util/automaton/TestDeterminism.java (added)
+++ lucene/dev/trunk/lucene/src/test/org/apache/lucene/util/automaton/TestDeterminism.java Mon May  3 23:23:57 2010
@@ -0,0 +1,72 @@
+package org.apache.lucene.util.automaton;
+
+/**
+ * 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.
+ */
+
+import java.util.Random;
+
+import org.apache.lucene.util.LuceneTestCase;
+
+/**
+ * Not thorough, but tries to test determinism correctness
+ * somewhat randomly.
+ */
+public class TestDeterminism extends LuceneTestCase {
+  private Random random;
+  
+  @Override
+  protected void setUp() throws Exception {
+    super.setUp();
+    random = newRandom();
+  }
+  
+  /** test a bunch of random regular expressions */
+  public void testRegexps() throws Exception {
+      for (int i = 0; i < 500; i++)
+        assertAutomaton(AutomatonTestUtil.randomRegexp(random).toAutomaton());
+  }
+  
+  private static void assertAutomaton(Automaton a) {
+    Automaton clone = a.clone();
+    // complement(complement(a)) = a
+    Automaton equivalent = BasicOperations.complement(BasicOperations.complement(a));
+    assertTrue(BasicOperations.sameLanguage(a, equivalent));
+    
+    // a union a = a
+    equivalent = BasicOperations.union(a, clone);
+    assertTrue(BasicOperations.sameLanguage(a, equivalent));
+    
+    // a intersect a = a
+    equivalent = BasicOperations.intersection(a, clone);
+    assertTrue(BasicOperations.sameLanguage(a, equivalent));
+    
+    // a minus a = empty
+    Automaton empty = BasicOperations.minus(a, clone);
+    assertTrue(BasicOperations.isEmpty(empty));
+    
+    // as long as don't accept the empty string
+    // then optional(a) - empty = a
+    if (!BasicOperations.run(a, "")) {
+      //System.out.println("test " + a);
+      Automaton optional = BasicOperations.optional(a);
+      //System.out.println("optional " + optional);
+      equivalent = BasicOperations.minus(optional, BasicAutomata.makeEmptyString());
+      //System.out.println("equiv " + equivalent);
+      assertTrue(BasicOperations.sameLanguage(a, equivalent));
+    }
+  } 
+}

Propchange: lucene/dev/trunk/lucene/src/test/org/apache/lucene/util/automaton/TestDeterminism.java
------------------------------------------------------------------------------
    svn:eol-style = native

Added: lucene/dev/trunk/lucene/src/test/org/apache/lucene/util/automaton/TestDeterminizeLexicon.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/test/org/apache/lucene/util/automaton/TestDeterminizeLexicon.java?rev=940671&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/src/test/org/apache/lucene/util/automaton/TestDeterminizeLexicon.java (added)
+++ lucene/dev/trunk/lucene/src/test/org/apache/lucene/util/automaton/TestDeterminizeLexicon.java Mon May  3 23:23:57 2010
@@ -0,0 +1,69 @@
+package org.apache.lucene.util.automaton;
+
+/**
+ * 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.
+ */
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Random;
+
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util._TestUtil;
+
+/**
+ * Not thorough, but tries to test determinism correctness
+ * somewhat randomly, by determinizing a huge random lexicon.
+ */
+public class TestDeterminizeLexicon extends LuceneTestCase {
+  private List<Automaton> automata = new ArrayList<Automaton>();
+  private List<String> terms = new ArrayList<String>();
+  private Random random;
+  
+  @Override
+  protected void setUp() throws Exception {
+    super.setUp();
+    random = newRandom();
+    for (int i = 0; i < 5000; i++) {
+      String randomString = _TestUtil.randomUnicodeString(random);
+      terms.add(randomString);
+      automata.add(BasicAutomata.makeString(randomString));
+    }
+  }
+
+  public void testLexicon() {
+    for (int i = 0; i < 3; i++) {
+      assertLexicon();
+    }
+  }
+  
+  public void assertLexicon() {
+    Collections.shuffle(automata, random);
+    final Automaton lex = BasicOperations.union(automata);
+    lex.determinize();
+    assertTrue(SpecialOperations.isFinite(lex));
+    for (String s : terms) {
+      assertTrue(BasicOperations.run(lex, s));
+    }
+    final ByteRunAutomaton lexByte = new ByteRunAutomaton(lex);
+    for (String s : terms) {
+      BytesRef termByte = new BytesRef(s);
+      assertTrue(lexByte.run(termByte.bytes, 0, termByte.length));
+    }
+  }
+}

Propchange: lucene/dev/trunk/lucene/src/test/org/apache/lucene/util/automaton/TestDeterminizeLexicon.java
------------------------------------------------------------------------------
    svn:eol-style = native

Modified: lucene/dev/trunk/lucene/src/test/org/apache/lucene/util/automaton/TestLevenshteinAutomata.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/test/org/apache/lucene/util/automaton/TestLevenshteinAutomata.java?rev=940671&r1=940670&r2=940671&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/test/org/apache/lucene/util/automaton/TestLevenshteinAutomata.java (original)
+++ lucene/dev/trunk/lucene/src/test/org/apache/lucene/util/automaton/TestLevenshteinAutomata.java Mon May  3 23:23:57 2010
@@ -21,10 +21,6 @@ import java.util.ArrayList;
 import java.util.List;
 
 import org.apache.lucene.util.LuceneTestCase;
-import org.apache.lucene.util.automaton.Automaton;
-import org.apache.lucene.util.automaton.BasicAutomata;
-import org.apache.lucene.util.automaton.BasicOperations;
-import org.apache.lucene.util.automaton.MinimizationOperations;
 
 public class TestLevenshteinAutomata extends LuceneTestCase {
  
@@ -79,11 +75,11 @@ public class TestLevenshteinAutomata ext
       switch(n) {
         case 0:
           // easy, matches the string itself
-          assertEquals(BasicAutomata.makeString(s), automata[0]);
+          assertTrue(BasicOperations.sameLanguage(BasicAutomata.makeString(s), automata[0]));
           break;
         case 1:
           // generate a lev1 naively, and check the accepted lang is the same.
-          assertEquals(naiveLev1(s), automata[1]);
+          assertTrue(BasicOperations.sameLanguage(naiveLev1(s), automata[1]));
           break;
         default:
           assertBruteForce(s, automata[n], n);
@@ -169,7 +165,7 @@ public class TestLevenshteinAutomata ext
   }
   
   private void assertBruteForce(String input, Automaton dfa, int distance) {
-    RunAutomaton ra = new RunAutomaton(dfa);
+    CharacterRunAutomaton ra = new CharacterRunAutomaton(dfa);
     int maxLen = input.length() + distance + 1;
     int maxNum = (int) Math.pow(2, maxLen);
     for (int i = 0; i < maxNum; i++) {

Added: lucene/dev/trunk/lucene/src/test/org/apache/lucene/util/automaton/TestUTF32ToUTF8.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/test/org/apache/lucene/util/automaton/TestUTF32ToUTF8.java?rev=940671&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/src/test/org/apache/lucene/util/automaton/TestUTF32ToUTF8.java (added)
+++ lucene/dev/trunk/lucene/src/test/org/apache/lucene/util/automaton/TestUTF32ToUTF8.java Mon May  3 23:23:57 2010
@@ -0,0 +1,192 @@
+package org.apache.lucene.util.automaton;
+
+/**
+ * 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.
+ */
+
+import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util._TestUtil;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.UnicodeUtil;
+
+import java.util.Random;
+
+public class TestUTF32ToUTF8 extends LuceneTestCase {
+  private Random random;
+  
+  @Override
+  protected void setUp() throws Exception {
+    super.setUp();
+    random = newRandom();
+  }
+
+  private static final int MAX_UNICODE = 0x10FFFF;
+
+  final BytesRef b = new BytesRef(4);
+
+  private boolean matches(ByteRunAutomaton a, int code) {
+    char[] chars = Character.toChars(code);
+    UnicodeUtil.UTF16toUTF8(chars, 0, chars.length, b);
+    return a.run(b.bytes, 0, b.length);
+  }
+
+  private void testOne(Random r, ByteRunAutomaton a, int startCode, int endCode, int iters) {
+
+    // Verify correct ints are accepted
+    for(int iter=0;iter<iters;iter++) {
+      // pick random code point in-range
+
+      final int code = _TestUtil.nextInt(r, startCode, endCode);
+      if ((code >= UnicodeUtil.UNI_SUR_HIGH_START && code <= UnicodeUtil.UNI_SUR_HIGH_END) |
+          (code >= UnicodeUtil.UNI_SUR_LOW_START && code <= UnicodeUtil.UNI_SUR_LOW_END)) {
+        iter--;
+        continue;
+      }
+      assertTrue("DFA for range " + startCode + "-" + endCode + " failed to match code=" + code, 
+                 matches(a, code));
+    }
+
+    // Verify invalid ints are not accepted
+    final int invalidRange = MAX_UNICODE - (endCode - startCode + 1);
+    if (invalidRange > 0) {
+      for(int iter=0;iter<iters;iter++) {
+        int x = _TestUtil.nextInt(r, 0, invalidRange-1);
+        final int code;
+        if (x >= startCode) {
+          code = endCode + 1 + x - startCode;
+        } else {
+          code = x;
+        }
+        if ((code >= UnicodeUtil.UNI_SUR_HIGH_START && code <= UnicodeUtil.UNI_SUR_HIGH_END) |
+            (code >= UnicodeUtil.UNI_SUR_LOW_START && code <= UnicodeUtil.UNI_SUR_LOW_END)) {
+          iter--;
+          continue;
+        }
+        assertFalse("DFA for range " + startCode + "-" + endCode + " matched invalid code=" + code,
+                    matches(a, code));
+                    
+      }
+    }
+  }
+
+  // Evenly picks random code point from the 4 "buckets"
+  // (bucket = same #bytes when encoded to utf8)
+  private int getCodeStart(Random r) {
+    switch(r.nextInt(4)) {
+    case 0:
+      return _TestUtil.nextInt(r, 0, 128);
+    case 1:
+      return _TestUtil.nextInt(r, 128, 2048);
+    case 2:
+      return _TestUtil.nextInt(r, 2048, 65536);
+    default:
+      return _TestUtil.nextInt(r, 65536, 1+MAX_UNICODE);
+    }
+  }
+
+  public void testRandomRanges() throws Exception {
+    final Random r = random;
+    int ITERS = 10;
+    int ITERS_PER_DFA = 100;
+    for(int iter=0;iter<ITERS;iter++) {
+      int x1 = getCodeStart(r);
+      int x2 = getCodeStart(r);
+      final int startCode, endCode;
+
+      if (x1 < x2) {
+        startCode = x1;
+        endCode = x2;
+      } else {
+        startCode = x2;
+        endCode = x1;
+      }
+      
+      final Automaton a = new Automaton();
+      final State end = new State();
+      end.setAccept(true);
+      a.getInitialState().addTransition(new Transition(startCode, endCode, end));
+      a.setDeterministic(true);
+
+      testOne(r, new ByteRunAutomaton(a), startCode, endCode, ITERS_PER_DFA);
+    }
+  }
+
+  public void testSpecialCase() {
+    RegExp re = new RegExp(".?");
+    Automaton automaton = re.toAutomaton();
+    CharacterRunAutomaton cra = new CharacterRunAutomaton(automaton);
+    ByteRunAutomaton bra = new ByteRunAutomaton(automaton);
+    // make sure character dfa accepts empty string
+    assertTrue(cra.isAccept(cra.getInitialState()));
+    assertTrue(cra.run(""));
+    assertTrue(cra.run(new char[0], 0, 0));
+
+    // make sure byte dfa accepts empty string
+    assertTrue(bra.isAccept(bra.getInitialState()));
+    assertTrue(bra.run(new byte[0], 0, 0));
+  }
+  
+  public void testSpecialCase2() throws Exception {
+    RegExp re = new RegExp(".+\u0775");
+    String input = "\ufadc\ufffd\ub80b\uda5a\udc68\uf234\u0056\uda5b\udcc1\ufffd\ufffd\u0775";
+    Automaton automaton = re.toAutomaton();
+    CharacterRunAutomaton cra = new CharacterRunAutomaton(automaton);
+    ByteRunAutomaton bra = new ByteRunAutomaton(automaton);
+
+    assertTrue(cra.run(input));
+    
+    byte[] bytes = input.getBytes("UTF-8");
+    assertTrue(bra.run(bytes, 0, bytes.length)); // this one fails!
+  }
+  
+  public void testSpecialCase3() throws Exception {
+    RegExp re = new RegExp("(\\鯺)*(.)*\\Ӕ");
+    String input = "\u5cfd\ufffd\ub2f7\u0033\ue304\u51d7\u3692\udb50\udfb3\u0576\udae2\udc62\u0053\u0449\u04d4";
+    Automaton automaton = re.toAutomaton();
+    CharacterRunAutomaton cra = new CharacterRunAutomaton(automaton);
+    ByteRunAutomaton bra = new ByteRunAutomaton(automaton);
+
+    assertTrue(cra.run(input));
+    
+    byte[] bytes = input.getBytes("UTF-8");
+    assertTrue(bra.run(bytes, 0, bytes.length));
+  }
+  
+  public void testRandomRegexes() throws Exception {
+    for (int i = 0; i < 250; i++)
+      assertAutomaton(AutomatonTestUtil.randomRegexp(random).toAutomaton());
+  }
+  
+  private void assertAutomaton(Automaton automaton) {
+    CharacterRunAutomaton cra = new CharacterRunAutomaton(automaton);
+    ByteRunAutomaton bra = new ByteRunAutomaton(automaton);
+    final BasicOperations.RandomAcceptedStrings ras = new BasicOperations.RandomAcceptedStrings(automaton);
+    
+    for (int i = 0; i < 1000; i++) {
+      final String string;
+      if (random.nextBoolean()) {
+        // likely not accepted
+        string = _TestUtil.randomUnicodeString(random);
+      } else {
+        // will be accepted
+        int[] codepoints = ras.getRandomAcceptedString(random);
+        string = new String(codepoints, 0, codepoints.length);
+      }
+      BytesRef bytesRef = new BytesRef(string);
+      assertEquals(cra.run(string), bra.run(bytesRef.bytes, 0, bytesRef.length));
+    }
+  }
+}

Propchange: lucene/dev/trunk/lucene/src/test/org/apache/lucene/util/automaton/TestUTF32ToUTF8.java
------------------------------------------------------------------------------
    svn:eol-style = native