You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hivemall.apache.org by my...@apache.org on 2021/04/22 14:53:17 UTC

[incubator-hivemall] branch master updated: Implement Korean text tokenizer

This is an automated email from the ASF dual-hosted git repository.

myui pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/incubator-hivemall.git


The following commit(s) were added to refs/heads/master by this push:
     new 0ee33d2  Implement Korean text tokenizer
0ee33d2 is described below

commit 0ee33d2f59a0c5e0a47abbecfd56fb864b831462
Author: Makoto Yui <my...@apache.org>
AuthorDate: Thu Apr 22 23:53:10 2021 +0900

    Implement Korean text tokenizer
    
    ## What changes were proposed in this pull request?
    
    Implement Korean text tokenizer
    
    ## What type of PR is it?
    
    Feature
    
    ## What is the Jira issue?
    
    https://issues.apache.org/jira/browse/HIVEMALL-307
    
    ## How was this patch tested?
    
    unit tests and manual tests on EMR
    
    ## How to use this feature?
    
    ```sql
    -- show version of lucene-analyzers-nori
    select tokenize_ko();
    > 8.8.2
    
    select tokenize_ko("소설 무궁화꽃이 피었습니다.");
    > ["소설","무궁","화","꽃","피"]
    
    select tokenize_ko("소설 무궁화꽃이 피었습니다.", null, "mixed");
    > ["소설","무궁화","무궁","화","꽃","피"]
    
    select tokenize_ko("소설 무궁화꽃이 피었습니다.", null, "discard", array("E", "VV"));
    > ["소설","무궁","화","꽃","이"]
    
    select tokenize_ko("Hello, world.", null, "none", array(), true);
    > ["h","e","l","l","o","w","o","r","l","d"]
    
    select tokenize_ko("Hello, world.", null, "none", array(), false);
    > ["hello","world"]
    
    select tokenize_ko("나는 C++ 언어를 프로그래밍 언어로 사랑한다.", null, "discard", array());
    > ["나","는","c","언어","를","프로그래밍","언어","로","사랑","하","ᆫ다"]
    
    select tokenize_ko("나는 C++ 언어를 프로그래밍 언어로 사랑한다.", array("C++"), "discard", array());
    > ["나","는","c++","언어","를","프로그래밍","언어","로","사랑","하","ᆫ다"]
    ```
    
    ## Checklist
    
    - [x] Did you apply source code formatter, i.e., `./bin/format_code.sh`, for your commit?
    - [x] Did you run system tests on Hive (or Spark)?
    
    Author: Makoto Yui <my...@apache.org>
    
    Closes #237 from myui/korean_tokenizer.
---
 dist/pom.xml                                       |   7 +-
 docs/Dockerfile                                    |  19 ++
 docs/gitbook/misc/funcs.md                         |   8 +
 docs/gitbook/misc/tokenizer.md                     |  52 ++-
 nlp/pom.xml                                        |  19 +-
 .../hivemall/nlp/tokenizer/KuromojiNEologdUDF.java |   2 +-
 .../java/hivemall/nlp/tokenizer/KuromojiUDF.java   |   2 +-
 .../java/hivemall/nlp/tokenizer/SmartcnUDF.java    |  17 +-
 .../java/hivemall/nlp/tokenizer/TokenizeKoUDF.java | 239 ++++++++++++++
 .../hivemall/nlp/tokenizer/tokenizer.properties    |   6 +-
 .../hivemall/nlp/tokenizer/SmartcnUDFTest.java     |  23 +-
 .../hivemall/nlp/tokenizer/TokenizeKoUDFTest.java  | 363 +++++++++++++++++++++
 resources/ddl/define-all-as-permanent.hive         |   3 +
 resources/ddl/define-all.hive                      |   3 +
 resources/ddl/define-all.spark                     |   3 +
 15 files changed, 745 insertions(+), 21 deletions(-)

diff --git a/dist/pom.xml b/dist/pom.xml
index 80104ac..f70b6fe 100644
--- a/dist/pom.xml
+++ b/dist/pom.xml
@@ -118,11 +118,12 @@
 									<include>com.clearspring.analytics:stream</include>
 									<!-- hivemall-nlp -->
 									<include>org.apache.hivemall:hivemall-nlp</include>
-									<include>org.apache.lucene:lucene-analyzers-kuromoji</include>
-									<include>org.apache.lucene:lucene-analyzers-smartcn</include>
-									<include>org.apache.lucene:lucene-analyzers-common</include>
 									<include>org.apache.lucene:lucene-core</include>
+									<include>org.apache.lucene:lucene-analyzers-common</include>
+									<include>org.apache.lucene:lucene-analyzers-kuromoji</include>
 									<include>io.github.myui:lucene-analyzers-kuromoji-neologd</include>
+									<include>org.apache.lucene:lucene-analyzers-smartcn</include>
+									<include>org.apache.lucene:lucene-analyzers-nori</include>
 									<!-- hivemall-xgboost -->
 									<include>org.apache.hivemall:hivemall-xgboost</include>
 									<include>io.github.myui:xgboost4j</include>
diff --git a/docs/Dockerfile b/docs/Dockerfile
index c764938..9b25cb8 100644
--- a/docs/Dockerfile
+++ b/docs/Dockerfile
@@ -1,3 +1,22 @@
+#
+# 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.
+#
+
 FROM node:8-buster-slim
 
 ARG GITBOOK_VERSION=3.2.3
diff --git a/docs/gitbook/misc/funcs.md b/docs/gitbook/misc/funcs.md
index 84f167c..b40019a 100644
--- a/docs/gitbook/misc/funcs.md
+++ b/docs/gitbook/misc/funcs.md
@@ -1075,6 +1075,14 @@ Reference: <a href="https://papers.nips.cc/paper/3848-adaptive-regularization-of
 
   ```
 
+- `tokenize_ko(String line [, const array<string> userDict, const string mode = "discard", const array<string> stopTags, boolean outputUnknownUnigrams])` - returns tokenized strings in array&lt;string&gt;
+  ```sql
+  select tokenize_ko("소설 무궁화꽃이 피었습니다.");
+
+  > ["소설","무궁","화","꽃","피"]
+
+  ```
+
 # Others
 
 - `hivemall_version()` - Returns the version of Hivemall
diff --git a/docs/gitbook/misc/tokenizer.md b/docs/gitbook/misc/tokenizer.md
index 78d9970..5ddc93e 100644
--- a/docs/gitbook/misc/tokenizer.md
+++ b/docs/gitbook/misc/tokenizer.md
@@ -174,14 +174,64 @@ Note that when `-pos` option is specified, `tokenize_ja` returns a struct record
 Chinese text tokenizer UDF uses [SmartChineseAnalyzer](https://lucene.apache.org/core/5_3_1/analyzers-smartcn/org/apache/lucene/analysis/cn/smart/SmartChineseAnalyzer.html). 
 
 The signature of the UDF is as follows:
+
 ```sql
 tokenize_cn(string line, optional const array<string> stopWords)
 ```
 
 Its basic usage is as follows:
+
 ```sql
 select tokenize_cn("Smartcn为Apache2.0协议的开源中文分词系统,Java语言编写,修改的中科院计算所ICTCLAS分词系统。");
-```
+
 > [smartcn, 为, apach, 2, 0, 协议, 的, 开源, 中文, 分词, 系统, java, 语言, 编写, 修改, 的, 中科院, 计算, 所, ictcla, 分词, 系统]
+```
 
 For detailed APIs, please refer Javadoc of [SmartChineseAnalyzer](https://lucene.apache.org/core/5_3_1/analyzers-smartcn/org/apache/lucene/analysis/cn/smart/SmartChineseAnalyzer.html) as well.
+
+## Korean Tokenizer
+
+Korean toknizer internally uses [lucene-analyzers-nori](analyzers-nori: Korean Morphological Analyzer) for tokenization.
+
+The signature of the UDF is as follows:
+
+```sql
+tokenize_ko(String line [,
+            const array<string> userDict,
+            const string mode = "discard",
+            const array<string> stopTags,
+            boolean outputUnknownUnigrams
+           ]) - returns tokenized strings in array<string>
+```
+
+> #### Note
+> For details options, please refer [Lucene API document](https://lucene.apache.org/core/8_8_2/analyzers-nori/org/apache/lucene/analysis/ko/KoreanAnalyzer.html). `none`, `discord` (default), or `mixed` are supported for the mode argument.
+
+See the following examples for the usage.
+
+```sql
+-- show version of lucene-analyzers-nori
+select tokenize_ko();
+> 8.8.2
+
+select tokenize_ko("소설 무궁화꽃이 피었습니다.");
+> ["소설","무궁","화","꽃","피"]
+
+select tokenize_ko("소설 무궁화꽃이 피었습니다.", null, "mixed");
+> ["소설","무궁화","무궁","화","꽃","피"]
+
+select tokenize_ko("소설 무궁화꽃이 피었습니다.", null, "discard", array("E", "VV"));
+> ["소설","무궁","화","꽃","이"]
+
+select tokenize_ko("Hello, world.", null, "none", array(), true);
+> ["h","e","l","l","o","w","o","r","l","d"]
+
+select tokenize_ko("Hello, world.", null, "none", array(), false);
+> ["hello","world"]
+
+select tokenize_ko("나는 C++ 언어를 프로그래밍 언어로 사랑한다.", null, "discard", array());
+> ["나","는","c","언어","를","프로그래밍","언어","로","사랑","하","ᆫ다"]
+
+select tokenize_ko("나는 C++ 언어를 프로그래밍 언어로 사랑한다.", array("C++"), "discard", array());
+> ["나","는","c++","언어","를","프로그래밍","언어","로","사랑","하","ᆫ다"]
+```
diff --git a/nlp/pom.xml b/nlp/pom.xml
index 55afab7..fdc8c4a 100644
--- a/nlp/pom.xml
+++ b/nlp/pom.xml
@@ -107,17 +107,23 @@
 			<scope>compile</scope>
 		</dependency>
 		<dependency>
-			<groupId>org.apache.lucene</groupId>
-			<artifactId>lucene-analyzers-smartcn</artifactId>
-			<version>${lucene.version}</version>
-			<scope>compile</scope>
-		</dependency>
-		<dependency>
 			<groupId>io.github.myui</groupId>
 			<artifactId>lucene-analyzers-kuromoji-neologd</artifactId>
 			<version>${lucene-analyzers-kuromoji-neologd.version}</version>
 			<scope>compile</scope>
 		</dependency>
+		<dependency>
+			<groupId>org.apache.lucene</groupId>
+			<artifactId>lucene-analyzers-smartcn</artifactId>
+			<version>${lucene.version}</version>
+			<scope>compile</scope>
+		</dependency>
+        <dependency>
+            <groupId>org.apache.lucene</groupId>
+            <artifactId>lucene-analyzers-nori</artifactId>
+            <version>${lucene.version}</version>
+            <scope>compile</scope>
+        </dependency>
 
 		<!-- test scope -->
 		<dependency>
@@ -135,6 +141,7 @@
 	</dependencies>
 
 	<build>
+		<!-- mvn resources:resources to process properties -->
 		<resources>
 			<resource>
 				<directory>src/main/resources</directory>
diff --git a/nlp/src/main/java/hivemall/nlp/tokenizer/KuromojiNEologdUDF.java b/nlp/src/main/java/hivemall/nlp/tokenizer/KuromojiNEologdUDF.java
index f3303a0..e7e4ace 100644
--- a/nlp/src/main/java/hivemall/nlp/tokenizer/KuromojiNEologdUDF.java
+++ b/nlp/src/main/java/hivemall/nlp/tokenizer/KuromojiNEologdUDF.java
@@ -175,7 +175,7 @@ public final class KuromojiNEologdUDF extends UDFWithOptions {
                 throw new HiveException("Failed to read tokenizer.properties");
             }
             return Collections.singletonList(
-                new Text(properties.getProperty("tokenizer_ja_neologd.version")));
+                new Text(properties.getProperty("tokenize_ja_neologd.version")));
         }
 
         if (_analyzer == null) {
diff --git a/nlp/src/main/java/hivemall/nlp/tokenizer/KuromojiUDF.java b/nlp/src/main/java/hivemall/nlp/tokenizer/KuromojiUDF.java
index 7902f60..8f05782 100644
--- a/nlp/src/main/java/hivemall/nlp/tokenizer/KuromojiUDF.java
+++ b/nlp/src/main/java/hivemall/nlp/tokenizer/KuromojiUDF.java
@@ -175,7 +175,7 @@ public final class KuromojiUDF extends UDFWithOptions {
                 throw new HiveException("Failed to read tokenizer.properties");
             }
             return Collections.singletonList(
-                new Text(properties.getProperty("tokenizer_ja.version")));
+                new Text(properties.getProperty("tokenize_ja.version")));
         }
 
         if (_analyzer == null) {
diff --git a/nlp/src/main/java/hivemall/nlp/tokenizer/SmartcnUDF.java b/nlp/src/main/java/hivemall/nlp/tokenizer/SmartcnUDF.java
index 93c8620..8bb5db9 100644
--- a/nlp/src/main/java/hivemall/nlp/tokenizer/SmartcnUDF.java
+++ b/nlp/src/main/java/hivemall/nlp/tokenizer/SmartcnUDF.java
@@ -24,7 +24,9 @@ import hivemall.utils.io.IOUtils;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.Collections;
 import java.util.List;
+import java.util.Properties;
 
 import javax.annotation.Nonnull;
 import javax.annotation.Nullable;
@@ -38,10 +40,10 @@ import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory;
 import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory;
 import org.apache.hadoop.io.Text;
+import org.apache.lucene.analysis.CharArraySet;
 import org.apache.lucene.analysis.TokenStream;
 import org.apache.lucene.analysis.cn.smart.SmartChineseAnalyzer;
 import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
-import org.apache.lucene.analysis.CharArraySet;
 
 @Description(name = "tokenize_cn", value = "_FUNC_(String line [, const list<string> stopWords])"
         + " - returns tokenized strings in array<string>")
@@ -55,7 +57,7 @@ public final class SmartcnUDF extends GenericUDF {
     @Override
     public ObjectInspector initialize(ObjectInspector[] arguments) throws UDFArgumentException {
         final int arglen = arguments.length;
-        if (arglen < 1 || arglen > 2) {
+        if (arglen > 2) {
             throw new UDFArgumentException(
                 "Invalid number of arguments for `tokenize_cn`: " + arglen);
         }
@@ -69,6 +71,17 @@ public final class SmartcnUDF extends GenericUDF {
 
     @Override
     public List<Text> evaluate(DeferredObject[] arguments) throws HiveException {
+        if (arguments.length == 0) {
+            final Properties properties = new Properties();
+            try {
+                properties.load(this.getClass().getResourceAsStream("tokenizer.properties"));
+            } catch (IOException e) {
+                throw new HiveException("Failed to read tokenizer.properties");
+            }
+            return Collections.singletonList(
+                new Text(properties.getProperty("tokenize_cn.version")));
+        }
+
         SmartChineseAnalyzer analyzer = _analyzer;
         if (analyzer == null) {
             CharArraySet stopwords = stopWords(_stopWordsArray);
diff --git a/nlp/src/main/java/hivemall/nlp/tokenizer/TokenizeKoUDF.java b/nlp/src/main/java/hivemall/nlp/tokenizer/TokenizeKoUDF.java
new file mode 100644
index 0000000..8c2a939
--- /dev/null
+++ b/nlp/src/main/java/hivemall/nlp/tokenizer/TokenizeKoUDF.java
@@ -0,0 +1,239 @@
+/*
+ * 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 hivemall.nlp.tokenizer;
+
+import hivemall.utils.hadoop.HiveUtils;
+import hivemall.utils.io.IOUtils;
+
+import java.io.IOException;
+import java.io.Reader;
+import java.io.StringReader;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Locale;
+import java.util.Properties;
+import java.util.Set;
+
+import javax.annotation.Nonnull;
+import javax.annotation.Nullable;
+
+import org.apache.hadoop.hive.ql.exec.Description;
+import org.apache.hadoop.hive.ql.exec.UDFArgumentException;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
+import org.apache.hadoop.hive.ql.udf.UDFType;
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDF;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory;
+import org.apache.hadoop.io.Text;
+import org.apache.lucene.analysis.TokenStream;
+import org.apache.lucene.analysis.ko.KoreanAnalyzer;
+import org.apache.lucene.analysis.ko.KoreanPartOfSpeechStopFilter;
+import org.apache.lucene.analysis.ko.KoreanTokenizer;
+import org.apache.lucene.analysis.ko.KoreanTokenizer.DecompoundMode;
+import org.apache.lucene.analysis.ko.POS;
+import org.apache.lucene.analysis.ko.dict.UserDictionary;
+import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
+
+@Description(name = "tokenize_ko",
+        value = "_FUNC_(String line [, const array<string> userDict, const string mode = \"discard\", const array<string> stopTags, boolean outputUnknownUnigrams])"
+                + " - returns tokenized strings in array<string>",
+        extended = "select tokenize_ko(\"소설 무궁화꽃이 피었습니다.\");\n" + "\n"
+                + "> [\"소설\",\"무궁\",\"화\",\"꽃\",\"피\"]\n")
+@UDFType(deterministic = true, stateful = false)
+public final class TokenizeKoUDF extends GenericUDF {
+
+    @Nullable
+    private UserDictionary userDict;
+
+    private DecompoundMode mode;
+    private Set<POS.Tag> stopTags;
+    private boolean outputUnknownUnigrams;
+
+    private transient KoreanAnalyzer analyzer;
+
+    @Override
+    public ObjectInspector initialize(ObjectInspector[] arguments) throws UDFArgumentException {
+        final int arglen = arguments.length;
+        if (arglen > 5) {
+            throw new UDFArgumentException(
+                "Invalid number of arguments for `tokenize_ko`: " + arglen);
+        }
+
+        this.userDict = (arglen >= 3) ? parseUserDict(arguments[1]) : null;
+        this.mode = (arglen >= 3) ? parseDecompoundMode(arguments[2])
+                : KoreanTokenizer.DEFAULT_DECOMPOUND;
+        this.stopTags = (arglen >= 4) ? parseStopTags(arguments[3])
+                : KoreanPartOfSpeechStopFilter.DEFAULT_STOP_TAGS;
+        this.outputUnknownUnigrams = (arglen >= 5) && HiveUtils.getConstBoolean(arguments[4]);
+
+        this.analyzer = null;
+
+        return ObjectInspectorFactory.getStandardListObjectInspector(
+            PrimitiveObjectInspectorFactory.writableStringObjectInspector);
+    }
+
+    @Override
+    public List<Text> evaluate(DeferredObject[] arguments) throws HiveException {
+        if (arguments.length == 0) {
+            final Properties properties = new Properties();
+            try {
+                properties.load(this.getClass().getResourceAsStream("tokenizer.properties"));
+            } catch (IOException e) {
+                throw new HiveException("Failed to read tokenizer.properties");
+            }
+            return Collections.singletonList(
+                new Text(properties.getProperty("tokenize_ko.version")));
+        }
+
+        if (analyzer == null) {
+            this.analyzer = new KoreanAnalyzer(userDict, mode, stopTags, outputUnknownUnigrams);
+        }
+
+        Object arg0 = arguments[0].get();
+        if (arg0 == null) {
+            return null;
+        }
+
+        String line = arg0.toString();
+
+        final List<Text> tokens = new ArrayList<Text>(32);
+        TokenStream stream = null;
+        try {
+            stream = analyzer.tokenStream("", line);
+            if (stream != null) {
+                analyzeTokens(stream, tokens);
+            }
+        } catch (IOException e) {
+            IOUtils.closeQuietly(analyzer);
+            throw new HiveException(e);
+        } finally {
+            IOUtils.closeQuietly(stream);
+        }
+        return tokens;
+    }
+
+    @Override
+    public void close() throws IOException {
+        IOUtils.closeQuietly(analyzer);
+    }
+
+    @Nullable
+    private static UserDictionary parseUserDict(@Nonnull final ObjectInspector oi)
+            throws UDFArgumentException {
+        if (HiveUtils.isVoidOI(oi)) {
+            return null;
+        }
+        final String[] array = HiveUtils.getConstStringArray(oi);
+        if (array == null) {
+            return null;
+        }
+        final int length = array.length;
+        if (length == 0) {
+            return null;
+        }
+        final StringBuilder builder = new StringBuilder();
+        for (int i = 0; i < length; i++) {
+            String row = array[i];
+            if (row != null) {
+                builder.append(row).append('\n');
+            }
+        }
+
+        final Reader reader = new StringReader(builder.toString());
+        try {
+            return UserDictionary.open(reader); // return null if empty
+        } catch (Throwable e) {
+            throw new UDFArgumentException(
+                "Failed to create user dictionary based on the given array<string>: "
+                        + builder.toString());
+        }
+    }
+
+    @Nonnull
+    private static DecompoundMode parseDecompoundMode(@Nonnull final ObjectInspector oi)
+            throws UDFArgumentException {
+        String arg = HiveUtils.getConstString(oi);
+        if (arg == null) {
+            return KoreanTokenizer.DEFAULT_DECOMPOUND;
+        }
+        final DecompoundMode mode;
+        try {
+            mode = DecompoundMode.valueOf(arg.toUpperCase(Locale.ENGLISH));
+        } catch (IllegalArgumentException e) {
+            final StringBuilder sb = new StringBuilder();
+            for (DecompoundMode v : DecompoundMode.values()) {
+                sb.append(v.toString()).append(", ");
+            }
+            throw new UDFArgumentException(
+                "Expected either " + sb.toString() + "but got an unexpected mode: " + arg);
+        }
+        return mode;
+    }
+
+    @Nonnull
+    private static Set<POS.Tag> parseStopTags(@Nonnull final ObjectInspector oi)
+            throws UDFArgumentException {
+        if (HiveUtils.isVoidOI(oi)) {
+            return KoreanPartOfSpeechStopFilter.DEFAULT_STOP_TAGS;
+        }
+        final String[] array = HiveUtils.getConstStringArray(oi);
+        if (array == null) {
+            return KoreanPartOfSpeechStopFilter.DEFAULT_STOP_TAGS;
+        }
+        final int length = array.length;
+        if (length == 0) {
+            return Collections.emptySet();
+        }
+        final Set<POS.Tag> stopTags = new HashSet<POS.Tag>(length);
+        for (int i = 0; i < length; i++) {
+            String s = array[i];
+            if (s != null) {
+                try {
+                    stopTags.add(POS.resolveTag(s));
+                } catch (IllegalArgumentException e) {
+                    throw new UDFArgumentException(
+                        "Unrecognized POS tag has been specified as a stop tag: " + e.getMessage());
+                }
+            }
+        }
+        return stopTags;
+    }
+
+    private static void analyzeTokens(@Nonnull TokenStream stream, @Nonnull List<Text> results)
+            throws IOException {
+        // instantiate an attribute placeholder once
+        CharTermAttribute termAttr = stream.getAttribute(CharTermAttribute.class);
+        stream.reset();
+
+        while (stream.incrementToken()) {
+            String term = termAttr.toString();
+            results.add(new Text(term));
+        }
+    }
+
+    @Override
+    public String getDisplayString(String[] children) {
+        return "tokenize_ko(" + Arrays.toString(children) + ')';
+    }
+
+}
diff --git a/nlp/src/main/resources/hivemall/nlp/tokenizer/tokenizer.properties b/nlp/src/main/resources/hivemall/nlp/tokenizer/tokenizer.properties
index 46e5acd..4118d1d 100644
--- a/nlp/src/main/resources/hivemall/nlp/tokenizer/tokenizer.properties
+++ b/nlp/src/main/resources/hivemall/nlp/tokenizer/tokenizer.properties
@@ -1,2 +1,4 @@
-tokenizer_ja.version=${lucene.version}
-tokenizer_ja_neologd.version=${lucene-analyzers-kuromoji-neologd.version}
+tokenize_ja.version=${lucene.version}
+tokenize_ja_neologd.version=${lucene-analyzers-kuromoji-neologd.version}
+tokenize_cn.version=${lucene.version}
+tokenize_ko.version=${lucene.version}
diff --git a/nlp/src/test/java/hivemall/nlp/tokenizer/SmartcnUDFTest.java b/nlp/src/test/java/hivemall/nlp/tokenizer/SmartcnUDFTest.java
index 342e48a..a5b7288 100644
--- a/nlp/src/test/java/hivemall/nlp/tokenizer/SmartcnUDFTest.java
+++ b/nlp/src/test/java/hivemall/nlp/tokenizer/SmartcnUDFTest.java
@@ -18,11 +18,11 @@
  */
 package hivemall.nlp.tokenizer;
 
+import hivemall.TestUtils;
+
 import java.io.IOException;
 import java.util.List;
 
-import hivemall.TestUtils;
-
 import org.apache.hadoop.hive.ql.exec.UDFArgumentException;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.udf.generic.GenericUDF;
@@ -37,6 +37,16 @@ import org.junit.Test;
 public class SmartcnUDFTest {
 
     @Test
+    public void testNoArgument() throws IOException, HiveException {
+        GenericUDF udf = new SmartcnUDF();
+        ObjectInspector[] argOIs = new ObjectInspector[0];
+        udf.initialize(argOIs);
+        Object result = udf.evaluate(new DeferredObject[0]);
+        Assert.assertNotNull(result);
+        udf.close();
+    }
+
+    @Test
     public void testOneArgument() throws UDFArgumentException, IOException {
         GenericUDF udf = new SmartcnUDF();
         ObjectInspector[] argOIs = new ObjectInspector[1];
@@ -74,10 +84,11 @@ public class SmartcnUDFTest {
             }
 
             @Override
-            public void prepare(int arg) throws HiveException {}
+            public void prepare(int arg) throws HiveException {
+            }
         };
         List<Text> tokens = udf.evaluate(args);
-        Assert.assertNotNull(tokens);
+        Assert.assertTrue(tokens.size() >= 2);
         udf.close();
     }
 
@@ -99,9 +110,11 @@ public class SmartcnUDFTest {
             }
 
             @Override
-            public void prepare(int arg) throws HiveException {}
+            public void prepare(int arg) throws HiveException {
+            }
         };
         List<Text> tokens = udf.evaluate(args);
+        Assert.assertTrue(tokens.size() >= 2);
 
         // serialization after evaluation
         serialized = TestUtils.serializeObjectByKryo(udf);
diff --git a/nlp/src/test/java/hivemall/nlp/tokenizer/TokenizeKoUDFTest.java b/nlp/src/test/java/hivemall/nlp/tokenizer/TokenizeKoUDFTest.java
new file mode 100644
index 0000000..5365de0
--- /dev/null
+++ b/nlp/src/test/java/hivemall/nlp/tokenizer/TokenizeKoUDFTest.java
@@ -0,0 +1,363 @@
+/*
+ * 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 hivemall.nlp.tokenizer;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.List;
+
+import org.apache.hadoop.hive.ql.exec.UDFArgumentException;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDF;
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDF.DeferredObject;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory;
+import org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo;
+import org.apache.hadoop.io.BooleanWritable;
+import org.apache.hadoop.io.Text;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+public class TokenizeKoUDFTest {
+
+    private TokenizeKoUDF udf;
+
+    @Before
+    public void setUp() {
+        this.udf = new TokenizeKoUDF();
+    }
+
+    @Test
+    public void testNoArgument() throws IOException, HiveException {
+        GenericUDF udf = new TokenizeKoUDF();
+        ObjectInspector[] argOIs = new ObjectInspector[0];
+        udf.initialize(argOIs);
+        Object result = udf.evaluate(new DeferredObject[0]);
+        Assert.assertNotNull(result);
+        udf.close();
+    }
+
+    @Test
+    public void test() throws HiveException, IOException {
+        ObjectInspector[] argOIs = new ObjectInspector[1];
+        argOIs[0] = PrimitiveObjectInspectorFactory.javaStringObjectInspector;
+        udf.initialize(argOIs);
+
+        GenericUDF.DeferredObject[] args = new GenericUDF.DeferredObject[1];
+        args[0] = new GenericUDF.DeferredObject() {
+            public Text get() throws HiveException {
+                return new Text("소설 무궁화꽃이 피었습니다.");
+            }
+
+            @Override
+            public void prepare(int arg) throws HiveException {
+            }
+        };
+        List<Text> tokens = udf.evaluate(args);
+
+        Assert.assertNotNull(tokens);
+        Assert.assertEquals(5, tokens.size());
+        Assert.assertEquals("소설 무궁 화 꽃 피", getString(tokens));
+
+        udf.close();
+    }
+
+    @Test
+    public void testNullUserList() throws HiveException, IOException {
+        ObjectInspector[] argOIs = new ObjectInspector[2];
+        // line
+        argOIs[0] = PrimitiveObjectInspectorFactory.javaStringObjectInspector;
+        // userDict
+        argOIs[1] = ObjectInspectorFactory.getStandardConstantListObjectInspector(
+            PrimitiveObjectInspectorFactory.javaStringObjectInspector, null);
+        udf.initialize(argOIs);
+
+        GenericUDF.DeferredObject[] args = new GenericUDF.DeferredObject[1];
+        args[0] = new GenericUDF.DeferredObject() {
+            public Text get() throws HiveException {
+                return new Text("I enjoy C++ programming.");
+            }
+
+            @Override
+            public void prepare(int arg) throws HiveException {
+            }
+        };
+        List<Text> tokens = udf.evaluate(args);
+
+        Assert.assertNotNull(tokens);
+        Assert.assertEquals(4, tokens.size());
+        Assert.assertEquals("i enjoy c programming", getString(tokens));
+
+        udf.close();
+    }
+
+    @Test
+    public void testNullMode() throws UDFArgumentException, IOException {
+        ObjectInspector[] argOIs = new ObjectInspector[3];
+        // line
+        argOIs[0] = PrimitiveObjectInspectorFactory.javaStringObjectInspector;
+        // userDict
+        argOIs[1] = ObjectInspectorFactory.getStandardConstantListObjectInspector(
+            PrimitiveObjectInspectorFactory.javaStringObjectInspector, null);
+        // mode
+        PrimitiveTypeInfo stringType = new PrimitiveTypeInfo();
+        stringType.setTypeName("string");
+        argOIs[2] = PrimitiveObjectInspectorFactory.getPrimitiveWritableConstantObjectInspector(
+            stringType, null);
+        udf.initialize(argOIs);
+        udf.close();
+    }
+
+    @Test
+    public void testMode() throws HiveException, IOException {
+        ObjectInspector[] argOIs = new ObjectInspector[3];
+        // line
+        argOIs[0] = PrimitiveObjectInspectorFactory.javaStringObjectInspector;
+        // userDict
+        argOIs[1] = ObjectInspectorFactory.getStandardConstantListObjectInspector(
+            PrimitiveObjectInspectorFactory.javaStringObjectInspector, null);
+        // mode
+        PrimitiveTypeInfo stringType = new PrimitiveTypeInfo();
+        stringType.setTypeName("string");
+        argOIs[2] = PrimitiveObjectInspectorFactory.getPrimitiveWritableConstantObjectInspector(
+            stringType, new Text("mixed"));
+        udf.initialize(argOIs);
+
+        GenericUDF.DeferredObject[] args = new GenericUDF.DeferredObject[1];
+        args[0] = new GenericUDF.DeferredObject() {
+            public Text get() throws HiveException {
+                return new Text("소설 무궁화꽃이 피었습니다.");
+            }
+
+            @Override
+            public void prepare(int arg) throws HiveException {
+            }
+        };
+        List<Text> tokens = udf.evaluate(args);
+
+        Assert.assertNotNull(tokens);
+        Assert.assertEquals(6, tokens.size());
+        Assert.assertEquals("소설 무궁화 무궁 화 꽃 피", getString(tokens));
+
+        udf.close();
+    }
+
+    @Test(expected = UDFArgumentException.class)
+    public void testInvalidMode() throws IOException, HiveException {
+        ObjectInspector[] argOIs = new ObjectInspector[3];
+        // line
+        argOIs[0] = PrimitiveObjectInspectorFactory.javaStringObjectInspector;
+        // userDict
+        argOIs[1] = ObjectInspectorFactory.getStandardConstantListObjectInspector(
+            PrimitiveObjectInspectorFactory.javaStringObjectInspector, null);
+        // mode
+        PrimitiveTypeInfo stringType = new PrimitiveTypeInfo();
+        stringType.setTypeName("string");
+        argOIs[2] = PrimitiveObjectInspectorFactory.getPrimitiveWritableConstantObjectInspector(
+            stringType, new Text("unsupported mode"));
+        udf.initialize(argOIs);
+        udf.close();
+    }
+
+    @Test
+    public void testNonnullUserList() throws HiveException, IOException {
+        ObjectInspector[] argOIs = new ObjectInspector[3];
+        // line
+        argOIs[0] = PrimitiveObjectInspectorFactory.javaStringObjectInspector;
+        // userDict
+        argOIs[1] = ObjectInspectorFactory.getStandardConstantListObjectInspector(
+            PrimitiveObjectInspectorFactory.javaStringObjectInspector, Arrays.asList("C++"));
+        // mode
+        PrimitiveTypeInfo stringType = new PrimitiveTypeInfo();
+        stringType.setTypeName("string");
+        argOIs[2] = PrimitiveObjectInspectorFactory.getPrimitiveWritableConstantObjectInspector(
+            stringType, new Text("mixed"));
+        udf.initialize(argOIs);
+
+        GenericUDF.DeferredObject[] args = new GenericUDF.DeferredObject[1];
+        args[0] = new GenericUDF.DeferredObject() {
+            public Text get() throws HiveException {
+                return new Text("I enjoy C++ programming.");
+            }
+
+            @Override
+            public void prepare(int arg) throws HiveException {
+            }
+        };
+        List<Text> tokens = udf.evaluate(args);
+
+        Assert.assertNotNull(tokens);
+        Assert.assertEquals(4, tokens.size());
+        Assert.assertEquals("i enjoy c++ programming", getString(tokens));
+
+        udf.close();
+    }
+
+    @Test
+    public void testStopTags() throws HiveException, IOException {
+        ObjectInspector[] argOIs = new ObjectInspector[4];
+        // line
+        argOIs[0] = PrimitiveObjectInspectorFactory.javaStringObjectInspector;
+        // userDict
+        argOIs[1] = ObjectInspectorFactory.getStandardConstantListObjectInspector(
+            PrimitiveObjectInspectorFactory.javaStringObjectInspector, null);
+        // mode
+        PrimitiveTypeInfo stringType = new PrimitiveTypeInfo();
+        stringType.setTypeName("string");
+        argOIs[2] = PrimitiveObjectInspectorFactory.getPrimitiveWritableConstantObjectInspector(
+            stringType, null);
+        // stopTags
+        argOIs[3] = ObjectInspectorFactory.getStandardConstantListObjectInspector(
+            PrimitiveObjectInspectorFactory.javaStringObjectInspector, Arrays.asList("E", "VV"));
+        udf.initialize(argOIs);
+
+        GenericUDF.DeferredObject[] args = new GenericUDF.DeferredObject[1];
+        args[0] = new GenericUDF.DeferredObject() {
+            public Text get() throws HiveException {
+                return new Text("소설 무궁화꽃이 피었습니다.");
+            }
+
+            @Override
+            public void prepare(int arg) throws HiveException {
+            }
+        };
+        List<Text> tokens = udf.evaluate(args);
+
+        Assert.assertNotNull(tokens);
+        Assert.assertEquals(5, tokens.size());
+        Assert.assertEquals("소설 무궁 화 꽃 이", getString(tokens));
+
+        udf.close();
+    }
+
+    @Test(expected = UDFArgumentException.class)
+    public void testInvalidStopTag() throws UDFArgumentException, IOException {
+        ObjectInspector[] argOIs = new ObjectInspector[4];
+        // line
+        argOIs[0] = PrimitiveObjectInspectorFactory.javaStringObjectInspector;
+        // userDict
+        argOIs[1] = ObjectInspectorFactory.getStandardConstantListObjectInspector(
+            PrimitiveObjectInspectorFactory.javaStringObjectInspector, null);
+        // mode
+        PrimitiveTypeInfo stringType = new PrimitiveTypeInfo();
+        stringType.setTypeName("string");
+        argOIs[2] = PrimitiveObjectInspectorFactory.getPrimitiveWritableConstantObjectInspector(
+            stringType, null);
+        // stopTags
+        argOIs[3] = ObjectInspectorFactory.getStandardConstantListObjectInspector(
+            PrimitiveObjectInspectorFactory.javaStringObjectInspector, Arrays.asList("E", "?"));
+        udf.initialize(argOIs);
+        udf.close();
+    }
+
+    @Test
+    public void testOutputUnknownUnigramsTrue() throws HiveException, IOException {
+        ObjectInspector[] argOIs = new ObjectInspector[5];
+        // line
+        argOIs[0] = PrimitiveObjectInspectorFactory.javaStringObjectInspector;
+        // userDict
+        argOIs[1] = ObjectInspectorFactory.getStandardConstantListObjectInspector(
+            PrimitiveObjectInspectorFactory.javaStringObjectInspector, null);
+        // mode
+        PrimitiveTypeInfo stringType = new PrimitiveTypeInfo();
+        stringType.setTypeName("string");
+        argOIs[2] = PrimitiveObjectInspectorFactory.getPrimitiveWritableConstantObjectInspector(
+            stringType, null);
+        // stopTags
+        argOIs[3] = ObjectInspectorFactory.getStandardConstantListObjectInspector(
+            PrimitiveObjectInspectorFactory.javaStringObjectInspector, null);
+        // outputUnknowUnigrams
+        PrimitiveTypeInfo booleanType = new PrimitiveTypeInfo();
+        booleanType.setTypeName("boolean");
+        argOIs[4] = PrimitiveObjectInspectorFactory.getPrimitiveWritableConstantObjectInspector(
+            booleanType, new BooleanWritable(true));
+        udf.initialize(argOIs);
+
+        GenericUDF.DeferredObject[] args = new GenericUDF.DeferredObject[1];
+        args[0] = new GenericUDF.DeferredObject() {
+            public Text get() throws HiveException {
+                return new Text("Hello, world.");
+            }
+
+            @Override
+            public void prepare(int arg) throws HiveException {
+            }
+        };
+        List<Text> tokens = udf.evaluate(args);
+
+        Assert.assertNotNull(tokens);
+        Assert.assertEquals(10, tokens.size());
+        Assert.assertEquals("h e l l o w o r l d", getString(tokens));
+
+        udf.close();
+    }
+
+    @Test
+    public void testOutputUnknownUnigramsFalse() throws HiveException, IOException {
+        ObjectInspector[] argOIs = new ObjectInspector[5];
+        // line
+        argOIs[0] = PrimitiveObjectInspectorFactory.javaStringObjectInspector;
+        // userDict
+        argOIs[1] = ObjectInspectorFactory.getStandardConstantListObjectInspector(
+            PrimitiveObjectInspectorFactory.javaStringObjectInspector, null);
+        // mode
+        PrimitiveTypeInfo stringType = new PrimitiveTypeInfo();
+        stringType.setTypeName("string");
+        argOIs[2] = PrimitiveObjectInspectorFactory.getPrimitiveWritableConstantObjectInspector(
+            stringType, null);
+        // stopTags
+        argOIs[3] = ObjectInspectorFactory.getStandardConstantListObjectInspector(
+            PrimitiveObjectInspectorFactory.javaStringObjectInspector, null);
+        // outputUnknowUnigrams
+        PrimitiveTypeInfo booleanType = new PrimitiveTypeInfo();
+        booleanType.setTypeName("boolean");
+        argOIs[4] = PrimitiveObjectInspectorFactory.getPrimitiveWritableConstantObjectInspector(
+            booleanType, new BooleanWritable(false));
+        udf.initialize(argOIs);
+
+        GenericUDF.DeferredObject[] args = new GenericUDF.DeferredObject[1];
+        args[0] = new GenericUDF.DeferredObject() {
+            public Text get() throws HiveException {
+                return new Text("Hello, world.");
+            }
+
+            @Override
+            public void prepare(int arg) throws HiveException {
+            }
+        };
+        List<Text> tokens = udf.evaluate(args);
+
+        Assert.assertNotNull(tokens);
+        Assert.assertEquals(2, tokens.size());
+        Assert.assertEquals("hello world", getString(tokens));
+
+        udf.close();
+    }
+
+    private static String getString(List<Text> tokens) {
+        StringBuilder sb = new StringBuilder();
+        for (Text token : tokens) {
+            sb.append(token.toString()).append(" ");
+        }
+        return sb.toString().trim();
+    }
+
+}
diff --git a/resources/ddl/define-all-as-permanent.hive b/resources/ddl/define-all-as-permanent.hive
index 209f161..91e0693 100644
--- a/resources/ddl/define-all-as-permanent.hive
+++ b/resources/ddl/define-all-as-permanent.hive
@@ -915,6 +915,9 @@ CREATE FUNCTION tokenize_ja_neologd as 'hivemall.nlp.tokenizer.KuromojiNEologdUD
 DROP FUNCTION IF EXISTS tokenize_cn;
 CREATE FUNCTION tokenize_cn as 'hivemall.nlp.tokenizer.SmartcnUDF' USING JAR '${hivemall_jar}';
 
+DROP FUNCTION IF EXISTS tokenize_ko;
+CREATE FUNCTION tokenize_ko as 'hivemall.nlp.tokenizer.TokenizeKoUDF' USING JAR '${hivemall_jar}';
+
 DROP FUNCTION IF EXISTS stoptags_exclude;
 CREATE FUNCTION stoptags_exclude as 'hivemall.nlp.tokenizer.StoptagsExcludeUDF' USING JAR '${hivemall_jar}';
 
diff --git a/resources/ddl/define-all.hive b/resources/ddl/define-all.hive
index 0e89507..c6c1a20 100644
--- a/resources/ddl/define-all.hive
+++ b/resources/ddl/define-all.hive
@@ -907,6 +907,9 @@ create temporary function tokenize_ja_neologd as 'hivemall.nlp.tokenizer.Kuromoj
 drop temporary function if exists tokenize_cn;
 create temporary function tokenize_cn as 'hivemall.nlp.tokenizer.SmartcnUDF';
 
+drop temporary function if exists tokenize_ko;
+create temporary function tokenize_ko as 'hivemall.nlp.tokenizer.TokenizeKoUDF';
+
 drop temporary function if exists stoptags_exclude;
 create temporary function stoptags_exclude as 'hivemall.nlp.tokenizer.StoptagsExcludeUDF';
 
diff --git a/resources/ddl/define-all.spark b/resources/ddl/define-all.spark
index 6eb2921..c9e7efc 100644
--- a/resources/ddl/define-all.spark
+++ b/resources/ddl/define-all.spark
@@ -893,6 +893,9 @@ sqlContext.sql("CREATE TEMPORARY FUNCTION tokenize_ja_neologd AS 'hivemall.nlp.t
 sqlContext.sql("DROP TEMPORARY FUNCTION IF EXISTS tokenize_cn")
 sqlContext.sql("CREATE TEMPORARY FUNCTION tokenize_cn AS 'hivemall.nlp.tokenizer.SmartcnUDF'")
 
+sqlContext.sql("DROP TEMPORARY FUNCTION IF EXISTS tokenize_ko")
+sqlContext.sql("CREATE TEMPORARY FUNCTION tokenize_ko AS 'hivemall.nlp.tokenizer.TokenizeKoUDF'")
+
 sqlContext.sql("DROP TEMPORARY FUNCTION IF EXISTS stoptags_exclude")
 sqlContext.sql("CREATE TEMPORARY FUNCTION stoptags_exclude AS 'hivemall.nlp.tokenizer.StoptagsExcludeUDF'")