You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucy.apache.org by nw...@apache.org on 2011/12/12 15:19:18 UTC

[lucy-commits] svn commit: r1213252 [1/4] - in /incubator/lucy/trunk: core/Lucy/Analysis/ core/Lucy/Test/Analysis/ devel/bin/ devel/conf/ modules/unicode/ucd/ perl/ perl/buildlib/Lucy/ perl/lib/Lucy/ perl/lib/Lucy/Analysis/ perl/t/ perl/t/core/

Author: nwellnhof
Date: Mon Dec 12 14:19:17 2011
New Revision: 1213252

URL: http://svn.apache.org/viewvc?rev=1213252&view=rev
Log:
LUCY-196 UAX #29 tokenizer

Merge branch LUCY-196-uax-tokenizer. This adds the new StandardTokenizer.

Added:
    incubator/lucy/trunk/core/Lucy/Analysis/StandardTokenizer.c
    incubator/lucy/trunk/core/Lucy/Analysis/StandardTokenizer.cfh
    incubator/lucy/trunk/core/Lucy/Test/Analysis/TestStandardTokenizer.c
    incubator/lucy/trunk/core/Lucy/Test/Analysis/TestStandardTokenizer.cfh
    incubator/lucy/trunk/devel/bin/UnicodeTable.pm
    incubator/lucy/trunk/devel/bin/gen_word_break_data.pl
    incubator/lucy/trunk/modules/unicode/ucd/
    incubator/lucy/trunk/modules/unicode/ucd/WordBreak.tab
    incubator/lucy/trunk/modules/unicode/ucd/WordBreakTest.json
    incubator/lucy/trunk/perl/lib/Lucy/Analysis/StandardTokenizer.pm
    incubator/lucy/trunk/perl/t/158-standard_tokenizer.t
    incubator/lucy/trunk/perl/t/core/158-standard_tokenizer.t
Modified:
    incubator/lucy/trunk/devel/conf/rat-excludes
    incubator/lucy/trunk/perl/MANIFEST
    incubator/lucy/trunk/perl/buildlib/Lucy/Build.pm
    incubator/lucy/trunk/perl/lib/Lucy/Test.pm

Added: incubator/lucy/trunk/core/Lucy/Analysis/StandardTokenizer.c
URL: http://svn.apache.org/viewvc/incubator/lucy/trunk/core/Lucy/Analysis/StandardTokenizer.c?rev=1213252&view=auto
==============================================================================
--- incubator/lucy/trunk/core/Lucy/Analysis/StandardTokenizer.c (added)
+++ incubator/lucy/trunk/core/Lucy/Analysis/StandardTokenizer.c Mon Dec 12 14:19:17 2011
@@ -0,0 +1,301 @@
+/* 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.
+ */
+
+#define C_LUCY_STANDARDTOKENIZER
+#define C_LUCY_TOKEN
+#include "Lucy/Util/ToolSet.h"
+
+#include "Lucy/Analysis/StandardTokenizer.h"
+#include "Lucy/Analysis/Token.h"
+#include "Lucy/Analysis/Inversion.h"
+
+/*
+ * We use a modified version of the Word_Break property defined in UAX #29.
+ * CR, LF, Newline and all undefined characters map to 0. WB_ASingle
+ * designates characters that are Alphabetic but are excluded from ALetter.
+ * WB_Extend_Format includes characters in both Extend and Format. The other
+ * WB_* values correspond to the standard properties.
+ *
+ * The tables are in a compressed format that uses a three-stage lookup
+ * scheme. They're generated with the perl script gen_word_break_tables.pl
+ * in devel/bin.
+ */
+
+#define WB_ASingle        1
+#define WB_ALetter        2
+#define WB_Numeric        3
+#define WB_Katakana       4
+#define WB_ExtendNumLet   5
+#define WB_Extend_Format  6
+#define WB_MidNumLet      7
+#define WB_MidLetter      8
+#define WB_MidNum         9
+
+#include "WordBreak.tab"
+
+typedef struct lucy_StringIter {
+    size_t byte_pos;
+    size_t char_pos;
+} lucy_StringIter;
+
+static int
+S_parse_single(const char *text, size_t len, lucy_StringIter *iter,
+               Inversion *inversion);
+
+static int
+S_parse_word(const char *text, size_t len, lucy_StringIter *iter,
+             int state, Inversion *inversion);
+
+static int
+S_wb_lookup(const char *ptr);
+
+static void
+S_iter_advance(const char *text, lucy_StringIter *iter);
+
+static int
+S_skip_extend_format(const char *text, size_t len, lucy_StringIter *iter);
+
+StandardTokenizer*
+StandardTokenizer_new() {
+    StandardTokenizer *self = (StandardTokenizer*)VTable_Make_Obj(STANDARDTOKENIZER);
+    return StandardTokenizer_init(self);
+}
+
+StandardTokenizer*
+StandardTokenizer_init(StandardTokenizer *self) {
+    Analyzer_init((Analyzer*)self);
+    return self;
+}
+
+Inversion*
+StandardTokenizer_transform(StandardTokenizer *self, Inversion *inversion) {
+    Inversion *new_inversion = Inversion_new(NULL);
+    Token *token;
+
+    while (NULL != (token = Inversion_Next(inversion))) {
+        StandardTokenizer_Tokenize_Str(self, token->text, token->len,
+                                    new_inversion);
+    }
+
+    return new_inversion;
+}
+
+Inversion*
+StandardTokenizer_transform_text(StandardTokenizer *self, CharBuf *text) {
+    Inversion *new_inversion = Inversion_new(NULL);
+    StandardTokenizer_Tokenize_Str(self, (char*)CB_Get_Ptr8(text),
+                                CB_Get_Size(text), new_inversion);
+    return new_inversion;
+}
+
+void
+StandardTokenizer_tokenize_str(StandardTokenizer *self, const char *text,
+                               size_t len, Inversion *inversion) {
+    if (len >= 1 && (uint8_t)text[len-1] >= 0xC0
+    ||  len >= 2 && (uint8_t)text[len-2] >= 0xE0
+    ||  len >= 3 && (uint8_t)text[len-3] >= 0xF0) {
+        THROW(ERR, "Invalid UTF-8 sequence");
+    }
+
+    lucy_StringIter iter = { 0, 0 };
+
+    while (iter.byte_pos < len) {
+        int wb = S_wb_lookup(text + iter.byte_pos);
+
+        while (wb >= WB_ASingle && wb <= WB_ExtendNumLet) {
+            if (wb == WB_ASingle) {
+                wb = S_parse_single(text, len, &iter, inversion);
+            }
+            else {
+                wb = S_parse_word(text, len, &iter, wb, inversion);
+            }
+            if (iter.byte_pos >= len) return;
+        }
+
+        S_iter_advance(text, &iter);
+    }
+}
+
+/*
+ * Parse a word consisting of a single codepoint followed by extend or
+ * format characters. Used for Alphabetic characters that don't have the
+ * ALetter word break property: ideographs, Hiragana, and "complex content".
+ * Advances the iterator and returns the word break property of the current
+ * character.
+ */
+static int
+S_parse_single(const char *text, size_t len, lucy_StringIter *iter,
+               Inversion *inversion) {
+    lucy_StringIter start = *iter;
+    int wb = S_skip_extend_format(text, len, iter);
+
+    Token *token = Token_new(text + start.byte_pos,
+                             iter->byte_pos - start.byte_pos,
+                             start.char_pos, iter->char_pos, 1.0f, 1);
+    Inversion_Append(inversion, token);
+
+    return wb;
+}
+
+/*
+ * Parse a word starting with an ALetter, Numeric or Katakana character.
+ * Advances the iterator and returns the word break property of the current
+ * character.
+ */
+static int
+S_parse_word(const char *text, size_t len, lucy_StringIter *iter,
+             int state, Inversion *inversion) {
+    int wb = -1;
+    lucy_StringIter start = *iter;
+    S_iter_advance(text, iter);
+    lucy_StringIter end = *iter;
+
+    while (iter->byte_pos < len) {
+        wb = S_wb_lookup(text + iter->byte_pos);
+
+        switch(wb) {
+          case WB_ALetter:
+          case WB_Numeric:
+            if (state == WB_Katakana) { goto word_break; }
+            break;
+          case WB_Katakana:
+            if (state == WB_ALetter || state == WB_Numeric) {
+                goto word_break;
+            }
+            break;
+          case WB_ExtendNumLet:
+            break;
+          case WB_Extend_Format:
+            // keep state
+            wb = state;
+            break;
+          case WB_MidNumLet:
+          case WB_MidLetter:
+          case WB_MidNum:
+            if (state == WB_ALetter && wb != WB_MidNum
+            ||  state == WB_Numeric && wb != WB_MidLetter) {
+                wb = S_skip_extend_format(text, len, iter);
+                if (wb == state) { break; }
+            }
+            goto word_break;
+          default:
+            goto word_break;
+        }
+
+        state = wb;
+        S_iter_advance(text, iter);
+        end = *iter;
+    }
+
+    Token *token;
+  word_break:
+    token = Token_new(text + start.byte_pos, end.byte_pos - start.byte_pos,
+                      start.char_pos, end.char_pos, 1.0f, 1);
+    Inversion_Append(inversion, token);
+
+    return wb;
+}
+
+/*
+ * Conceptually, the word break property table is split into rows that
+ * contain 64 columns and planes that contain 64 rows (not to be confused
+ * the 65,536 character Unicode planes). So bits 0-5 of a code point contain
+ * the column index into a row, bits 6-11 contain the row index into a plane,
+ * and bits 12-20 contain the plane index.
+ *
+ * To save space, identical rows are merged so the row table contains only
+ * unique rows and the plane table contains row indices remapped to row ids.
+ * Then, identical planes are merged, and a plane map table is created with
+ * plane indices remapped to plane ids.
+ *
+ * The row and plane tables are simple one-dimensional arrays created by
+ * concatenating all unique rows and planes. So looking up an entry can be
+ * done by left shifting the id and ORing the index.
+ */
+
+#define WB_TABLE_LOOKUP(table, id, index) table [ ((id) << 6) | (index) ]
+
+static int
+S_wb_lookup(const char *ptr) {
+    uint8_t start = *(uint8_t*)ptr++;
+
+    if (start < 0x80) { return wb_ascii[start]; }
+
+    size_t plane_id, row_index;
+
+    if (start < 0xE0) {
+        if (start < 0xC0) {
+            THROW(ERR, "Invalid UTF-8 sequence");
+        }
+        // two byte sequence
+        // 110rrrrr 10cccccc
+        plane_id  = 0;
+        row_index = start & 0x1F;
+    }
+    else {
+        size_t plane_index;
+        if (start < 0xF0) {
+            // three byte sequence
+            // 1110pppp 10rrrrrr 10cccccc
+            plane_index = start & 0x0F;
+        }
+        else {
+            // four byte sequence
+            // 11110ppp 10pppppp 10rrrrrr 10cccccc
+            plane_index = ((start & 0x07) << 6) | (*ptr++ & 0x3F);
+        }
+        if (plane_index >= WB_PLANE_MAP_SIZE) { return 0; }
+        plane_id  = wb_plane_map[plane_index];
+        row_index = *ptr++ & 0x3F;
+    }
+
+    size_t row_id = WB_TABLE_LOOKUP(wb_planes, plane_id, row_index);
+    size_t column_index = *ptr++ & 0x3F;
+    return WB_TABLE_LOOKUP(wb_rows, row_id, column_index);
+}
+
+static void
+S_iter_advance(const char *text, lucy_StringIter *iter) {
+    iter->byte_pos += StrHelp_UTF8_COUNT[*(uint8_t*)(text + iter->byte_pos)];
+    iter->char_pos += 1;
+}
+
+/*
+ * Advances the iterator skipping over Extend and Format characters.
+ * Returns the word break property of the current character.
+ */
+static int
+S_skip_extend_format(const char *text, size_t len, lucy_StringIter *iter) {
+    int wb = -1;
+
+    do {
+        S_iter_advance(text, iter);
+        if (iter->byte_pos >= len) { break; }
+        wb = S_wb_lookup(text + iter->byte_pos);
+    } while (wb == WB_Extend_Format);
+
+    return wb;
+}
+
+bool_t
+StandardTokenizer_equals(StandardTokenizer *self, Obj *other) {
+    StandardTokenizer *const twin = (StandardTokenizer*)other;
+    if (twin == self)                        { return true; }
+    if (!Obj_Is_A(other, STANDARDTOKENIZER)) { return false; }
+    return true;
+}
+
+

Added: incubator/lucy/trunk/core/Lucy/Analysis/StandardTokenizer.cfh
URL: http://svn.apache.org/viewvc/incubator/lucy/trunk/core/Lucy/Analysis/StandardTokenizer.cfh?rev=1213252&view=auto
==============================================================================
--- incubator/lucy/trunk/core/Lucy/Analysis/StandardTokenizer.cfh (added)
+++ incubator/lucy/trunk/core/Lucy/Analysis/StandardTokenizer.cfh Mon Dec 12 14:19:17 2011
@@ -0,0 +1,57 @@
+/* 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.
+ */
+
+parcel Lucy;
+
+/** Split a string into tokens.
+ *
+ * Generically, "tokenizing" is a process of breaking up a string into an
+ * array of "tokens".  For instance, the string "three blind mice" might be
+ * tokenized into "three", "blind", "mice".
+ *
+ * Lucy::Analysis::StandardTokenizer breaks up the text at the word
+ * boundaries defined in Unicode Standard Annex #29. It then returns those
+ * words that start with an alphabetic or numeric character.
+ */
+class Lucy::Analysis::StandardTokenizer
+    inherits Lucy::Analysis::Analyzer {
+
+    inert incremented StandardTokenizer*
+    new();
+
+    /** Constructor.  Takes no arguments.
+     */
+    public inert StandardTokenizer*
+    init(StandardTokenizer *self);
+
+    public incremented Inversion*
+    Transform(StandardTokenizer *self, Inversion *inversion);
+
+    public incremented Inversion*
+    Transform_Text(StandardTokenizer *self, CharBuf *text);
+
+    /** Tokenize the supplied string and add any Tokens generated to the
+     * supplied Inversion.
+     */
+    void
+    Tokenize_Str(StandardTokenizer *self, const char *text, size_t len,
+                 Inversion *inversion);
+
+    public bool_t
+    Equals(StandardTokenizer *self, Obj *other);
+}
+
+

Added: incubator/lucy/trunk/core/Lucy/Test/Analysis/TestStandardTokenizer.c
URL: http://svn.apache.org/viewvc/incubator/lucy/trunk/core/Lucy/Test/Analysis/TestStandardTokenizer.c?rev=1213252&view=auto
==============================================================================
--- incubator/lucy/trunk/core/Lucy/Test/Analysis/TestStandardTokenizer.c (added)
+++ incubator/lucy/trunk/core/Lucy/Test/Analysis/TestStandardTokenizer.c Mon Dec 12 14:19:17 2011
@@ -0,0 +1,130 @@
+/* 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.
+ */
+
+#define C_LUCY_TESTSTANDARDTOKENIZER
+#include "Lucy/Util/ToolSet.h"
+
+#include "Lucy/Test.h"
+#include "Lucy/Test/Analysis/TestStandardTokenizer.h"
+#include "Lucy/Analysis/StandardTokenizer.h"
+#include "Lucy/Store/FSFolder.h"
+#include "Lucy/Util/Json.h"
+
+static void
+test_Dump_Load_and_Equals(TestBatch *batch) {
+    StandardTokenizer *tokenizer = StandardTokenizer_new();
+    Obj *dump  = StandardTokenizer_Dump(tokenizer);
+    StandardTokenizer *clone = (StandardTokenizer*)StandardTokenizer_Load(tokenizer, dump);
+
+    TEST_TRUE(batch,
+              StandardTokenizer_Equals(tokenizer, (Obj*)clone),
+              "Dump => Load round trip");
+
+    DECREF(tokenizer);
+    DECREF(dump);
+    DECREF(clone);
+}
+
+static void
+test_tokenizer(TestBatch *batch) {
+    StandardTokenizer *tokenizer = StandardTokenizer_new();
+
+    ZombieCharBuf *word = ZCB_WRAP_STR(
+        " ."
+        "tha\xCC\x82t's"
+        ":"
+        "1,02\xC2\xADZ4.38"
+        "\xE0\xB8\x81\xC2\xAD\xC2\xAD"
+        "\xF0\xA0\x80\x80"
+        "a"
+        "/",
+        35);
+    VArray *got = StandardTokenizer_Split(tokenizer, (CharBuf*)word);
+    CharBuf *token = (CharBuf*)VA_Fetch(got, 0);
+    TEST_TRUE(batch,
+              token
+              && CB_Is_A(token, CHARBUF)
+              && CB_Equals_Str(token, "tha\xcc\x82t's", 8),
+              "Token: %s", CB_Get_Ptr8(token));
+    token = (CharBuf*)VA_Fetch(got, 1);
+    TEST_TRUE(batch,
+              token
+              && CB_Is_A(token, CHARBUF)
+              && CB_Equals_Str(token, "1,02\xC2\xADZ4.38", 11),
+              "Token: %s", CB_Get_Ptr8(token));
+    token = (CharBuf*)VA_Fetch(got, 2);
+    TEST_TRUE(batch,
+              token
+              && CB_Is_A(token, CHARBUF)
+              && CB_Equals_Str(token, "\xE0\xB8\x81\xC2\xAD\xC2\xAD", 7),
+              "Token: %s", CB_Get_Ptr8(token));
+    token = (CharBuf*)VA_Fetch(got, 3);
+    TEST_TRUE(batch,
+              token
+              && CB_Is_A(token, CHARBUF)
+              && CB_Equals_Str(token, "\xF0\xA0\x80\x80", 4),
+              "Token: %s", CB_Get_Ptr8(token));
+    token = (CharBuf*)VA_Fetch(got, 4);
+    TEST_TRUE(batch,
+              token
+              && CB_Is_A(token, CHARBUF)
+              && CB_Equals_Str(token, "a", 1),
+              "Token: %s", CB_Get_Ptr8(token));
+    DECREF(got);
+
+    CharBuf  *path           = CB_newf("modules");
+    FSFolder *modules_folder = FSFolder_new(path);
+    if (!FSFolder_Check(modules_folder)) {
+        DECREF(modules_folder);
+        CB_setf(path, "../modules");
+        modules_folder = FSFolder_new(path);
+        if (!FSFolder_Check(modules_folder)) {
+            THROW(ERR, "Can't open modules folder");
+        }
+    }
+    CB_setf(path, "unicode/ucd/WordBreakTest.json");
+    VArray *tests = (VArray*)Json_slurp_json((Folder*)modules_folder, path);
+    if (!tests) { RETHROW(Err_get_error()); }
+
+    for (uint32_t i = 0, max = VA_Get_Size(tests); i < max; i++) {
+        Hash *test = (Hash*)VA_Fetch(tests, i);
+        CharBuf *text = (CharBuf*)Hash_Fetch_Str(test, "text", 4);
+        VArray *wanted = (VArray*)Hash_Fetch_Str(test, "words", 5);
+        VArray *got = StandardTokenizer_Split(tokenizer, text);
+        TEST_TRUE(batch, VA_Equals(wanted, (Obj*)got), "UCD test #%d", i + 1);
+        DECREF(got);
+    }
+
+    DECREF(tests);
+    DECREF(modules_folder);
+    DECREF(path);
+
+    DECREF(tokenizer);
+}
+
+void
+TestStandardTokenizer_run_tests() {
+    TestBatch *batch = TestBatch_new(984);
+
+    TestBatch_Plan(batch);
+
+    test_Dump_Load_and_Equals(batch);
+    test_tokenizer(batch);
+
+    DECREF(batch);
+}
+
+

Added: incubator/lucy/trunk/core/Lucy/Test/Analysis/TestStandardTokenizer.cfh
URL: http://svn.apache.org/viewvc/incubator/lucy/trunk/core/Lucy/Test/Analysis/TestStandardTokenizer.cfh?rev=1213252&view=auto
==============================================================================
--- incubator/lucy/trunk/core/Lucy/Test/Analysis/TestStandardTokenizer.cfh (added)
+++ incubator/lucy/trunk/core/Lucy/Test/Analysis/TestStandardTokenizer.cfh Mon Dec 12 14:19:17 2011
@@ -0,0 +1,24 @@
+/* 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.
+ */
+
+parcel Lucy;
+
+inert class Lucy::Test::Analysis::TestStandardTokenizer {
+    inert void
+    run_tests();
+}
+
+

Added: incubator/lucy/trunk/devel/bin/UnicodeTable.pm
URL: http://svn.apache.org/viewvc/incubator/lucy/trunk/devel/bin/UnicodeTable.pm?rev=1213252&view=auto
==============================================================================
--- incubator/lucy/trunk/devel/bin/UnicodeTable.pm (added)
+++ incubator/lucy/trunk/devel/bin/UnicodeTable.pm Mon Dec 12 14:19:17 2011
@@ -0,0 +1,426 @@
+# 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.
+
+package UnicodeTable;
+use strict;
+
+=head1 NAME
+
+UnicodeTable - Create compressed Unicode tables for C programs
+
+=head1 SYNOPSIS
+
+    my $table = UnicodeTable->read(
+        filename => $filename,
+        type     => 'Enumerated',
+        map      => \%map,
+    );
+
+    my $comp = $table->compress($shift);
+
+    $comp->dump;
+
+=head1 DESCRIPTION
+
+This module creates compressed tables used to lookup Unicode properties
+in C programs. To compress a table, it's split into blocks of a fixed
+size. Identical blocks are discovered and only unique blocks are written to
+the compressed table. An additional map table is created to map original
+block indices to block ids.
+
+The map tables can then be compressed again using the same algorithm.
+
+Powers of two are used as block sizes, so the table indices to lookup values
+can be computed using bit operations.
+
+=head1 METHODS
+
+=head2 new
+
+    my $table = UnicodeTable->new(
+        values    => \@values,
+        default   => $default,
+        max       => $max,
+        shift     => $shift,
+        map_table => $map_table,
+    );
+
+\@values is an arrayref with the table values, $max is the maximum value.
+The default value for undefined table entries is $default or 0.
+$shift and $map_table are used for compressed tables.
+
+=cut
+
+sub new {
+    my $class = shift;
+
+    my $opts = @_ == 1 ? $_[0] : {@_};
+    my $self = bless( {}, $class );
+
+    for my $name (qw(values default max shift map_table)) {
+        $self->{$name} = $opts->{$name};
+    }
+
+    $self->{default} = 0
+        if !defined( $self->{default} );
+    $self->{mask} = ( 1 << $self->{shift} ) - 1
+        if defined( $self->{shift} );
+
+    return $self;
+}
+
+=head2 read
+
+    my $table = UnicodeTable->table(
+        filename => $filename,
+        type     => $type,
+        map      => \%map,
+        default  => $default,
+    );
+
+Reads a table from a Unicode data text file. $type is either 'Enumerated'
+or 'Boolean'. \%map is a hashref that maps property values to integers.
+For booleans, these integers are ORed. $default is the default value passed
+to L<new>.
+
+=cut
+
+sub read {
+    my $class = shift;
+
+    my $opts = @_ == 1 ? $_[0] : {@_};
+    my $max = 0;
+    my @values;
+
+    my $filename = $opts->{filename};
+    die('filename missing') if !defined($filename);
+    my $type = $opts->{type} or die('type missing');
+    my $map  = $opts->{map}  or die('map missing');
+    $type = lc($type);
+
+    open( my $file, '<', $filename )
+        or die("$filename: $!\n");
+
+    while ( my $line = $file->getline ) {
+        $line =~ s/\s*(#.*)?\z//s;
+        next if $line eq '';
+        my ( $chars, $prop ) = split( /\s*;\s*/, $line );
+        my $val = $map->{$prop};
+
+        if ( !defined($val) ) {
+            if ( $type eq 'boolean' ) {
+                next;
+            }
+            else {
+                die("unknown property '$prop'");
+            }
+        }
+
+        $max = $val if $val > $max;
+
+        if ( $chars =~ /^[0-9A-Fa-f]+\z/ ) {
+            my $i = hex($chars);
+            if ( $type eq 'boolean' ) {
+                $values[$i] |= $val;
+            }
+            else {
+                $values[$i] = $val;
+            }
+        }
+        elsif ( $chars =~ /^(\w+)\.\.(\w+)\z/ ) {
+            my ( $l, $r ) = ( hex($1), hex($2) );
+            die("invalid range '$chars'") if $l > $r;
+
+            for ( my $i = $l; $i <= $r; ++$i ) {
+                if ( $type eq 'boolean' ) {
+                    $values[$i] |= $val;
+                }
+                else {
+                    $values[$i] = $val;
+                }
+            }
+        }
+        else {
+            die("invalid range '$chars'");
+        }
+    }
+
+    close($file);
+
+    return $class->new(
+        values  => \@values,
+        default => $opts->{default},
+        max     => $max,
+    );
+}
+
+=head2 shift
+
+=head2 mask
+
+=head2 max
+
+=head2 map_table
+
+Accessors
+
+=cut
+
+sub shift {
+    return $_[0]->{shift};
+}
+
+sub mask {
+    return $_[0]->{mask};
+}
+
+sub max {
+    return $_[0]->{max};
+}
+
+sub map_table {
+    return $_[0]->{map_table};
+}
+
+=head2 set
+
+    $table->set($i, $value);
+
+Set entry at index $i to $value. Don't use with compressed tables.
+
+=cut
+
+sub set {
+    my ( $self, $i, $value ) = @_;
+    $self->{values}[$i] = $value;
+    $self->{max} = $value if $value > $self->{max};
+}
+
+=head2 size
+
+    my $size = $table->size;
+
+Storage size of the table in bytes.
+
+=cut
+
+sub size {
+    my $self = CORE::shift;
+
+    my $max = $self->{max};
+    my $bytes = $max < 0x100 ? 1 : $max < 0x10000 ? 2 : 4;
+
+    return @{ $self->{values} } * $bytes;
+}
+
+=head2 lookup
+
+    my $value = $table->lookup($i);
+
+Lookup value at index $i. Also works with compressed tables.
+
+=cut
+
+sub lookup {
+    my ( $self, $i ) = @_;
+
+    my $map_table = $self->{map_table};
+
+    if ($map_table) {
+        my $shift = $self->{shift};
+        my $id    = $map_table->lookup( $i >> $shift );
+        my $j     = ( $id << $shift ) | ( $i & $self->{mask} );
+        return $self->{values}->[$j];
+    }
+    else {
+        my $val = $self->{values}->[$i];
+        return $self->{default} if !defined($val);
+        return $val;
+    }
+}
+
+=head2 compress
+
+    my $compressed_table = $table->compress($shift);
+
+Returns a compressed version of this table which is linked to a second
+map table. Blocks of size (1 << $shift) are used.
+
+=cut
+
+sub compress {
+    my ( $self, $shift ) = @_;
+
+    my $values      = $self->{values};
+    my $default     = $self->{default};
+    my $block_size  = 1 << $shift;
+    my $block_count = 0;
+    my ( @compressed, @map_values, %block_ids );
+
+    for ( my $start = 0; $start < @$values; $start += $block_size ) {
+        my @block;
+
+        for ( my $i = $start; $i < $start + $block_size; ++$i ) {
+            my $val = $values->[$i];
+            $val = $default if !defined($val);
+            push( @block, $val );
+        }
+
+        my $str = join( '|', @block );
+        my $block_id = $block_ids{$str};
+
+        if ( !defined($block_id) ) {
+            $block_id = $block_count++;
+            $block_ids{$str} = $block_id;
+            push( @compressed, @block );
+        }
+
+        push( @map_values, $block_id );
+    }
+
+    # find default for map table
+
+    my @default_block;
+
+    for ( my $i = 0; $i < $block_size; ++$i ) {
+        push( @default_block, $default );
+    }
+
+    my $str = join( '|', @default_block );
+    my $default_block_id = $block_ids{$str};
+
+    if ( !defined($default_block_id) ) {
+        $default_block_id = $block_count++;
+        push( @compressed, @default_block );
+    }
+
+    my $map_table = UnicodeTable->new(
+        values  => \@map_values,
+        default => $default_block_id,
+        max     => $block_count - 1,
+    );
+
+    return UnicodeTable->new(
+        values    => \@compressed,
+        default   => $default,
+        max       => $self->{max},
+        shift     => $shift,
+        map_table => $map_table,
+    );
+}
+
+=head2 compress_map
+
+    my $map_table = $table->compress_map($shift);
+
+Compress the map table of a table for multi stage lookup. Returns the
+compressed map table.
+
+=cut
+
+sub compress_map {
+    my ( $self, $shift ) = @_;
+
+    my $comp = $self->{map_table}->compress($shift);
+    $self->{map_table} = $comp;
+
+    return $comp;
+}
+
+=head2 dump
+
+    $table->dump($file, $name);
+
+Dump the table as C code to filehandle $file. The table name is $name.
+
+=cut
+
+sub dump {
+    my ( $self, $file, $name ) = @_;
+
+    my $values  = $self->{values};
+    my $size    = @$values;
+    my $uc_name = uc($name);
+
+    print $file (<<"EOF") if $self->{shift};
+#define ${uc_name}_SHIFT $self->{shift}
+#define ${uc_name}_MASK  $self->{mask}
+EOF
+    print $file (<<"EOF");
+#define ${uc_name}_SIZE  $size
+
+EOF
+
+    my $max           = $self->{max};
+    my $bits          = $max < 0x100 ? 8 : $max < 0x10000 ? 16 : 32;
+    my $pad           = length($max);
+    my $vals_per_line = int( 76 / ( $pad + 2 ) );
+
+    print $file ("static const uint${bits}_t $name\[$size] = {\n");
+
+    my $i = 0;
+
+    while ( $i < $size ) {
+        printf $file ( "    \%${pad}d", $values->[$i] );
+
+        my $max = $i + $vals_per_line;
+        $max = $size if $max > $size;
+
+        while ( ++$i < $max ) {
+            printf $file ( ", \%${pad}d", $values->[$i] );
+        }
+
+        print $file (',') if $i < $size;
+        print $file ("\n");
+    }
+
+    print $file ("};\n");
+}
+
+sub calc_sizes {
+    my ( $self, $range2, $range1 ) = @_;
+
+    for ( my $shift2 = $range2->[0]; $shift2 <= $range2->[1]; ++$shift2 ) {
+        my $comp      = $self->compress($shift2);
+        my $map_table = $comp->map_table;
+        my $size3     = $comp->size;
+
+        for ( my $shift1 = $range1->[0]; $shift1 <= $range1->[1]; ++$shift1 )
+        {
+            my $comp_map_table = $map_table->compress($shift1);
+
+            my $size1 = $comp_map_table->map_table->size;
+            my $size2 = $comp_map_table->size;
+
+            printf(
+                "shift %2d %2d: %6d + %6d + %6d = %7d bytes, %4d %4d\n",
+                $shift1,                         $shift2,
+                $size1,                          $size2,
+                $size3,                          $size1 + $size2 + $size3,
+                $comp_map_table->map_table->max, $comp_map_table->max,
+            );
+        }
+
+        print("\n");
+    }
+}
+
+=head1 AUTHOR
+
+Nick Wellnhofer <we...@aevum.de>
+
+=cut
+
+1;

Added: incubator/lucy/trunk/devel/bin/gen_word_break_data.pl
URL: http://svn.apache.org/viewvc/incubator/lucy/trunk/devel/bin/gen_word_break_data.pl?rev=1213252&view=auto
==============================================================================
--- incubator/lucy/trunk/devel/bin/gen_word_break_data.pl (added)
+++ incubator/lucy/trunk/devel/bin/gen_word_break_data.pl Mon Dec 12 14:19:17 2011
@@ -0,0 +1,245 @@
+#!/usr/bin/perl
+
+# 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.
+
+=head1 NAME
+
+gen_word_break_data.pl - Generate word break table and tests
+
+=head1 SYNOPSIS
+
+    perl gen_word_break_data.pl [-c] UCD_SRC_DIR
+
+=head1 DESCRIPTION
+
+This script generates the tables to lookup Unicode word break properties
+for the StandardTokenizer. It also converts the word break test suite in
+the UCD to JSON.
+
+UCD_SRC_DIR should point to a directory containing the files
+WordBreakProperty.txt, WordBreakTest.txt, and DerivedCoreProperties.txt from
+the Unicode Character Database available at
+L<http://www.unicode.org/Public/6.0.0/ucd/>.
+
+=head1 OUTPUT FILES
+
+    modules/unicode/ucd/WordBreak.tab
+    modules/unicode/ucd/WordBreakTest.json
+
+=head1 OPTIONS
+
+=head2 -c
+
+Show total table size for different shift values
+
+=cut
+
+use strict;
+
+use Getopt::Std;
+use JSON;
+use UnicodeTable;
+
+my $output_dir     = '../../modules/unicode/ucd';
+my $table_filename = "$output_dir/WordBreak.tab";
+my $tests_filename = "$output_dir/WordBreakTest.json";
+
+my %wb_map = (
+    CR           => 0,
+    LF           => 0,
+    Newline      => 0,
+    ALetter      => 2,
+    Numeric      => 3,
+    Katakana     => 4,
+    ExtendNumLet => 5,
+    Extend       => 6,
+    Format       => 6,
+    MidNumLet    => 7,
+    MidLetter    => 8,
+    MidNum       => 9,
+);
+
+my %opts;
+if ( !getopts( 'c', \%opts ) || @ARGV != 1 ) {
+    print STDERR ("Usage: $0 [-c] UCD_SRC_DIR\n");
+    exit;
+}
+
+my $src_dir = $ARGV[0];
+
+my $wb = UnicodeTable->read(
+    filename => "$src_dir/WordBreakProperty.txt",
+    type     => 'Enumerated',
+    map      => \%wb_map,
+);
+my $alpha = UnicodeTable->read(
+    filename => "$src_dir/DerivedCoreProperties.txt",
+    type     => 'Boolean',
+    map      => { Alphabetic => 1 },
+);
+
+# Set characters in Alphabetic but not in Word_Break to WB_ASingle = 1
+for ( my $i = 0; $i < 0x30000; ++$i ) {
+    if ( !$wb->lookup($i) && $alpha->lookup($i) ) {
+        $wb->set( $i, 1 );
+    }
+}
+
+if ( $opts{c} ) {
+    $wb->calc_sizes( [ 2, 6 ], [ 3, 9 ] );
+    exit;
+}
+
+# Optimize for UTF-8
+my $row_shift   = 6;
+my $plane_shift = 6;
+
+my $wb_ascii = UnicodeTable->new(
+    table => [],
+    max   => 0,
+);
+
+for ( my $i = 0; $i < 0x80; ++$i ) {
+    $wb_ascii->set( $i, $wb->lookup($i) );
+}
+
+my $wb_rows      = $wb->compress($row_shift);
+my $wb_planes    = $wb_rows->compress_map($plane_shift);
+my $wb_plane_map = $wb_planes->map_table;
+
+# test compressed table
+
+for ( my $i = 0; $i < 0x110000; ++$i ) {
+    my $v1 = $wb->lookup($i);
+    my $v2 = $wb_rows->lookup($i);
+    die("test for code point $i failed, want $v1, got $v2")
+        if $v1 != $v2;
+}
+
+# dump tables
+
+open( my $out_file, '>', $table_filename )
+    or die("$table_filename: $!\n");
+
+print $out_file (<DATA>);
+
+$wb_ascii->dump( $out_file, 'wb_ascii' );
+print $out_file ("\n");
+$wb_plane_map->dump( $out_file, 'wb_plane_map' );
+print $out_file ("\n");
+$wb_planes->dump( $out_file, 'wb_planes' );
+print $out_file ("\n");
+$wb_rows->dump( $out_file, 'wb_rows' );
+
+close($out_file);
+
+# convert UCD test suite
+
+open( my $in_file, '<', "$src_dir/WordBreakTest.txt" )
+    or die("$src_dir/WordBreakTest.txt: $!\n");
+binmode( $in_file, ':utf8' );
+
+my @tests;
+
+while (<$in_file>) {
+    s/\s*(#.*)?\z//s;
+    next if $_ eq '';
+    my @items = split(/\s+/);
+    my $word  = '';
+    my $text  = '';
+    my @words;
+
+    for ( my $i = 0; $i + 1 < @items; $i += 2 ) {
+        my ( $break, $code ) = ( $items[$i], hex( $items[ $i + 1 ] ) );
+        my $chr = chr($code);
+        $text .= $chr;
+
+        if ( $break eq "\xF7" ) {    # division sign
+            if ( $word ne '' ) {
+                push( @words, $word );
+                $word = '';
+            }
+
+            my $wb = $wb->lookup($code);
+            $word = $chr if $wb >= 1 && $wb <= 5;
+        }
+        elsif ( $break eq "\xD7" ) {    # multiplication sign
+            $word .= $chr if $word ne '';
+        }
+        else {
+            die("invalid break character '$break'");
+        }
+    }
+
+    push( @words, $word ) if $word ne '';
+
+    push(
+        @tests,
+        {   text  => $text,
+            words => \@words,
+        }
+    );
+}
+
+close($in_file);
+
+open( $out_file, '>', $tests_filename )
+    or die("$tests_filename: $!\n");
+print $out_file ( JSON->new->utf8->pretty->encode( \@tests ) );
+close($out_file);
+
+__DATA__
+/*
+
+This file is generated with devel/bin/gen_word_break_data.pl. DO NOT EDIT!
+The contents of this file are derived from the Unicode Character Database,
+version 6.0.0, available from http://www.unicode.org/Public/6.0.0/ucd/.
+The Unicode copyright and permission notice follows.
+
+Copyright (c) 1991-2011 Unicode, Inc. All rights reserved. Distributed under
+the Terms of Use in http://www.unicode.org/copyright.html.
+
+Permission is hereby granted, free of charge, to any person obtaining a copy of
+the Unicode data files and any associated documentation (the "Data Files") or
+Unicode software and any associated documentation (the "Software") to deal in
+the Data Files or Software without restriction, including without limitation
+the rights to use, copy, modify, merge, publish, distribute, and/or sell copies
+of the Data Files or Software, and to permit persons to whom the Data Files or
+Software are furnished to do so, provided that (a) the above copyright
+notice(s) and this permission notice appear with all copies of the Data Files
+or Software, (b) both the above copyright notice(s) and this permission notice
+appear in associated documentation, and (c) there is clear notice in each
+modified Data File or in the Software as well as in the documentation
+associated with the Data File(s) or Software that the data or software has been
+modified.
+
+THE DATA FILES AND SOFTWARE ARE PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND,
+EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF
+MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT OF THIRD
+PARTY RIGHTS. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR HOLDERS INCLUDED IN
+THIS NOTICE BE LIABLE FOR ANY CLAIM, OR ANY SPECIAL INDIRECT OR CONSEQUENTIAL
+DAMAGES, OR ANY DAMAGES WHATSOEVER RESULTING FROM LOSS OF USE, DATA OR PROFITS,
+WHETHER IN AN ACTION OF CONTRACT, NEGLIGENCE OR OTHER TORTIOUS ACTION, ARISING
+OUT OF OR IN CONNECTION WITH THE USE OR PERFORMANCE OF THE DATA FILES OR
+SOFTWARE.
+
+Except as contained in this notice, the name of a copyright holder shall not be
+used in advertising or otherwise to promote the sale, use or other dealings in
+these Data Files or Software without prior written authorization of the
+copyright holder.
+
+*/
+

Modified: incubator/lucy/trunk/devel/conf/rat-excludes
URL: http://svn.apache.org/viewvc/incubator/lucy/trunk/devel/conf/rat-excludes?rev=1213252&r1=1213251&r2=1213252&view=diff
==============================================================================
--- incubator/lucy/trunk/devel/conf/rat-excludes (original)
+++ incubator/lucy/trunk/devel/conf/rat-excludes Mon Dec 12 14:19:17 2011
@@ -48,8 +48,11 @@ modules/analysis/snowstem/source/test/te
 # This file is autogenerated, as indicated in the comment at the top.
 modules/analysis/snowstop/source/snowball_stoplists.c
 
-# The Unicode license as applied to utf8proc was dealt with in LEGAL-110.
+# The Unicode license as applied to utf8proc and the Unicode Character Database
+# was dealt with in LEGAL-110.
 modules/unicode/utf8proc/utf8proc_data.c
+modules/unicode/ucd/WordBreak.tab
+modules/unicode/ucd/WordBreakTest.json
 
 # For whatever reason, RAT does not recognize the MIT license of utf8proc.h
 # and utf8proc.c.