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 2016/12/01 05:25:25 UTC

[01/50] [abbrv] incubator-hivemall git commit: Added a gitbook userguide

Repository: incubator-hivemall
Updated Branches:
  refs/heads/master [created] 72d6a629f


http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/370e2aa3/docs/gitbook/tips/hadoop_tuning.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/tips/hadoop_tuning.md b/docs/gitbook/tips/hadoop_tuning.md
new file mode 100644
index 0000000..a6c1854
--- /dev/null
+++ b/docs/gitbook/tips/hadoop_tuning.md
@@ -0,0 +1,79 @@
+# Prerequisites 
+
+Please refer the following guides for Hadoop tuning:
+
+* http://hadoopbook.com/
+* http://www.slideshare.net/cloudera/mr-perf
+
+---
+# Mapper-side configuration
+_Mapper configuration is important for hivemall when training runs on mappers (e.g., when using rand_amplify())._
+
+```
+mapreduce.map.java.opts="-Xmx2048m -XX:+PrintGCDetails" (YARN)
+mapred.map.child.java.opts="-Xmx2048m -XX:+PrintGCDetails" (MR v1)
+
+mapreduce.task.io.sort.mb=1024 (YARN)
+io.sort.mb=1024 (MR v1)
+```
+
+Hivemall can use at max 1024MB in the above case.
+> mapreduce.map.java.opts - mapreduce.task.io.sort.mb = 2048MB - 1024MB = 1024MB
+
+Moreover, other Hadoop components consumes memory spaces. It would be about 1024MB * 0.5 or so is available for Hivemall. We recommend to set at least -Xmx2048m for a mapper.
+ 
+So, make `mapreduce.map.java.opts - mapreduce.task.io.sort.mb` as large as possible.
+
+# Reducer-side configuration
+_Reducer configuration is important for hivemall when training runs on reducers (e.g., when using amplify())._
+
+```
+mapreduce.reduce.java.opts="-Xmx2048m -XX:+PrintGCDetails" (YARN)
+mapred.reduce.child.java.opts="-Xmx2048m -XX:+PrintGCDetails" (MR v1)
+
+mapreduce.reduce.shuffle.input.buffer.percent=0.6 (YARN)
+mapred.reduce.shuffle.input.buffer.percent=0.6 (MR v1)
+
+-- mapreduce.reduce.input.buffer.percent=0.2 (YARN)
+-- mapred.job.reduce.input.buffer.percent=0.2 (MR v1)
+```
+
+Hivemall can use at max 820MB in the above case.
+> mapreduce.reduce.java.opts * (1 - mapreduce.reduce.input.buffer.percent) = 2048 * (1 - 0.6) \u2248 820 MB
+
+Moreover, other Hadoop components consumes memory spaces. It would be about 820MB * 0.5 or so is available for Hivemall. We recommend to set at least -Xmx2048m for a reducer.
+
+So, make `mapreduce.reduce.java.opts * (1 - mapreduce.reduce.input.buffer.percent)` as large as possible.
+
+---
+# Formula to estimate consumed memory in Hivemall
+
+For a dense model, the consumed memory in Hivemall is as follows:
+```
+feature_dimensions (2^24 by the default) * 4 bytes (float) * 2 (iff covariance is calculated) * 1.2 (heuristics)
+```
+> 2^24 * 4 bytes * 2 * 1.2 \u2248 161MB
+
+When [SpaceEfficientDenseModel](https://github.com/myui/hivemall/blob/master/src/main/java/hivemall/io/SpaceEfficientDenseModel.java) is used, the formula changes as follows:
+```
+feature_dimensions (assume here 2^25) * 2 bytes (short) * 2 (iff covariance is calculated) * 1.2 (heuristics)
+```
+> 2^25 * 2 bytes * 2 * 1.2 \u2248 161MB
+
+Note: Hivemall uses a [sparse representation](https://github.com/myui/hivemall/blob/master/src/main/java/hivemall/io/SparseModel.java) of prediction model (using a hash table) by the default. Use "[-densemodel](https://github.com/myui/hivemall/blob/master/src/main/java/hivemall/LearnerBaseUDTF.java#L87)" option to use a dense model.
+
+# Execution Engine of Hive
+
+We recommend to use Apache Tez for execute engine of Hive for Hivemall queries.
+
+```sql
+set mapreduce.framework.name=yarn-tez;
+set hive.execution.engine=tez;
+```
+
+You can use the plain old MapReduce by setting following setting:
+
+```sql
+set mapreduce.framework.name=yarn;
+set hive.execution.engine=mr;
+```
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/370e2aa3/docs/gitbook/tips/mixserver.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/tips/mixserver.md b/docs/gitbook/tips/mixserver.md
new file mode 100644
index 0000000..631557c
--- /dev/null
+++ b/docs/gitbook/tips/mixserver.md
@@ -0,0 +1,68 @@
+In this page, we will explain how to use model mixing on Hivemall. The model mixing is useful for a better prediction performance and faster convergence in training classifiers. 
+
+<!--
+You can find a brief explanation of the internal design of MIX protocol in [this slide](http://www.slideshare.net/myui/hivemall-mix-internal).
+-->
+
+Prerequisite
+============
+
+* Hivemall v0.3 or later
+
+We recommend to use Mixing in a cluster with fast networking. The current standard GbE is enough though.
+
+Running Mix Server
+===================
+
+First, put the following files on server(s) that are accessible from Hadoop worker nodes:
+* [target/hivemall-mixserv.jar](https://github.com/myui/hivemall/releases)
+* [bin/run_mixserv.sh](https://github.com/myui/hivemall/raw/master/bin/run_mixserv.sh)
+
+_Caution: hivemall-mixserv.jar is large in size and thus only used for Mix servers._
+
+```sh
+# run a Mix Server
+./run_mixserv.sh
+```
+
+We assume in this example that Mix servers are running on host01, host03 and host03.
+The default port used by Mix server is 11212 and the port is configurable through "-port" option of run_mixserv.sh. 
+
+See [MixServer.java](https://github.com/myui/hivemall/blob/master/mixserv/src/main/java/hivemall/mix/server/MixServer.java#L90) to get detail of the Mix server options.
+
+We recommended to use multiple MIX servers to get better MIX throughput (3-5 or so would be enough for normal cluster size). The MIX protocol of Hivemall is *horizontally scalable* by adding MIX server nodes.
+
+Using Mix Protocol through Hivemall
+===================================
+
+[Install Hivemall](https://github.com/myui/hivemall/wiki/Installation) on Hive.
+
+_Make sure that [hivemall-with-dependencies.jar](https://github.com/myui/hivemall/raw/master/target/hivemall-with-dependencies.jar) is used for installation. The jar contains minimum requirement jars (netty,jsr305) for running Hivemall on Hive._
+
+Now, we explain that how to use mixing in [an example using KDD2010a dataset](https://github.com/myui/hivemall/wiki/KDD2010a-classification).
+
+Enabling the mixing on Hivemall is simple as follows:
+```sql
+use kdd2010;
+
+create table kdd10a_pa1_model1 as
+select 
+ feature,
+ cast(voted_avg(weight) as float) as weight
+from 
+ (select 
+     train_pa1(addBias(features),label,"-mix host01,host02,host03") as (feature,weight)
+  from 
+     kdd10a_train_x3
+ ) t 
+group by feature;
+```
+
+All you have to do is just adding "*-mix*" training option as seen in the above query.
+
+The effect of model mixing
+===========================
+
+In my experience, the MIX improved the prediction accuracy of the above KDD2010a PA1 training on a 32 nodes cluster from 0.844835019263103 (w/o mix) to 0.8678096499719774 (w/ mix).
+
+The overhead of using the MIX protocol is *almost negligible* because the MIX communication is efficiently handled using asynchronous non-blocking I/O. Furthermore, the training time could be improved on certain settings because of the faster convergence due to mixing. 
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/370e2aa3/docs/gitbook/tips/rand_amplify.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/tips/rand_amplify.md b/docs/gitbook/tips/rand_amplify.md
new file mode 100644
index 0000000..4df124e
--- /dev/null
+++ b/docs/gitbook/tips/rand_amplify.md
@@ -0,0 +1,103 @@
+This article explains *amplify* technique that is useful for improving prediction score.
+
+Iterations are mandatory in machine learning (e.g., in [stochastic gradient descent](http://en.wikipedia.org/wiki/Stochastic_gradient_descent)) to get good prediction models. However, MapReduce is known to be not suited for iterative algorithms because IN/OUT of each MapReduce job is through HDFS.
+
+In this example, we show how Hivemall deals with this problem. We use [KDD Cup 2012, Track 2 Task](https://github.com/myui/hivemall/wiki/KDDCup-2012-track-2-CTR-prediction-dataset) as an example.
+
+**WARNING**: rand_amplify() is supported in v0.2-beta1 and later.
+
+---
+# Amplify training examples in Map phase and shuffle them in Reduce phase
+Hivemall provides the **amplify** UDTF to enumerate iteration effects in machine learning without several MapReduce steps. 
+
+The amplify function returns multiple rows for each row.
+The first argument ${xtimes} is the multiplication factor.  
+In the following examples, the multiplication factor is set to 3.
+
+```sql
+set hivevar:xtimes=3;
+
+create or replace view training_x3
+as
+select 
+  * 
+from (
+select
+   amplify(${xtimes}, *) as (rowid, label, features)
+from  
+   training_orcfile
+) t
+CLUSTER BY rand();
+```
+
+In the above example, the  [CLUSTER BY](https://cwiki.apache.org/confluence/display/Hive/LanguageManual+SortBy#LanguageManualSortBy-SyntaxofClusterByandDistributeBy) clause distributes Map outputs to reducers using a random key for the distribution key. And then, the input records of the reducer is randomly shuffled.
+
+The multiplication of records and  the random shuffling has a similar effect to iterations.
+So, we recommend users to use an amplified view for training as follows:
+
+```sql
+create table lr_model_x3 
+as
+select 
+ feature,
+ cast(avg(weight) as float) as weight
+from 
+ (select 
+     logress(features,label) as (feature,weight)
+  from 
+     training_x3
+ ) t 
+group by feature;
+```
+
+The above query is executed by 2 MapReduce jobs as shown below:
+![amplifier](https://dl.dropboxusercontent.com/u/13123103/hivemall/amplify.png)
+[Here](https://dl.dropboxusercontent.com/u/13123103/hivemall/amplify_plan.txt) is the actual plan generated by the Hive.
+
+Using *trainning_x3*  instead of the plain training table results in higher and better AUC (0.746214) in [this](https://github.com/myui/hivemall/wiki/KDDCup-2012-track-2-CTR-prediction-(regression\)) example.
+
+A problem in amplify() is that the shuffle (copy) and merge phase of the stage 1 could become a bottleneck.
+When the training table is so large that involves 100 Map tasks, the merge operator needs to merge at least 100 files by (external) merge sort! 
+
+Note that the actual bottleneck is not M/R iterations but shuffling training instance. Iteration without shuffling (as in [the Spark example](http://spark.incubator.apache.org/examples.html)) causes very slow convergence and results in requiring more iterations. Shuffling cannot be avoided even in iterative MapReduce variants.
+
+![amplify elapsed](https://dl.dropboxusercontent.com/u/13123103/hivemall/amplify_elapsed.png)
+
+---
+# Amplify and shuffle training examples in each Map task
+
+To deal with large training data, Hivemall provides **rand_amplify** UDTF that randomly shuffles input rows in a Map task.
+The rand_amplify UDTF outputs rows in a random order when the local buffer specified by ${shufflebuffersize} is filled.
+
+With rand_amplify(), the view definition of training_x3 becomes as follows:
+```sql
+set hivevar:shufflebuffersize=1000;
+
+create or replace view training_x3
+as
+select
+   rand_amplify(${xtimes}, ${shufflebuffersize}, *) as (rowid, label, features)
+from  
+   training_orcfile;
+```
+
+The training query is executed as follows:
+![Random amplify](https://dl.dropboxusercontent.com/u/13123103/hivemall/randamplify.png)  
+[Here](https://dl.dropboxusercontent.com/u/13123103/hivemall/randamplify_plan.txt) is the actual query plan.
+
+The map-local multiplication and shuffling has no bottleneck in the merge phase and the query is efficiently executed within a single MapReduce job.
+
+![rand_amplify elapsed ](https://dl.dropboxusercontent.com/u/13123103/hivemall/randamplify_elapsed.png)
+
+Using *rand_amplify* results in a better AUC (0.743392) in [this](https://github.com/myui/hivemall/wiki/KDDCup-2012-track-2-CTR-prediction-(regression\)) example.
+
+---
+# Conclusion
+
+We recommend users to use *amplify()* for small training inputs and to use *rand_amplify()* for large training inputs to get a better accuracy in a reasonable training time.
+
+| Method     | ELAPSED TIME (sec) | AUC |
+|:-----------|--------------------|----:|
+| Plain | 89.718 | 0.734805 |
+| amplifier+clustered by | 479.855  | 0.746214 |
+| rand_amplifier | 116.424 | 0.743392 |
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/370e2aa3/docs/gitbook/tips/rowid.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/tips/rowid.md b/docs/gitbook/tips/rowid.md
new file mode 100644
index 0000000..c43aa74
--- /dev/null
+++ b/docs/gitbook/tips/rowid.md
@@ -0,0 +1,31 @@
+```sql
+CREATE TABLE xxx
+AS
+SELECT 
+  regexp_replace(reflect('java.util.UUID','randomUUID'), '-', '') as rowid,
+  *
+FROM
+  ..;
+```
+
+Another option to generate rowid is to use row_number(). 
+However, the query execution would become too slow for large dataset because the rowid generation is executed on a single reducer.
+```sql
+CREATE TABLE xxx
+AS
+select 
+  row_number() over () as rowid, 
+  * 
+from a9atest;
+```
+
+***
+# Rowid generator provided in Hivemall v0.2 or later
+You can use [rowid() function](https://github.com/myui/hivemall/blob/master/src/main/java/hivemall/tools/mapred/RowIdUDF.java) to generate an unique rowid in Hivemall v0.2 or later.
+```sql
+select
+  rowid() as rowid, -- returns ${task_id}-${sequence_number}
+  *
+from 
+  xxx
+```
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/370e2aa3/docs/gitbook/tips/rt_prediction.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/tips/rt_prediction.md b/docs/gitbook/tips/rt_prediction.md
new file mode 100644
index 0000000..3ac4fb6
--- /dev/null
+++ b/docs/gitbook/tips/rt_prediction.md
@@ -0,0 +1,234 @@
+Hivemall provides a batch learning scheme that builds prediction models on Apache Hive.
+The learning process itself is a batch process; however, an online/real-time prediction can be achieved by carrying a prediction on a transactional relational DBMS.
+
+In this article, we explain how to run a real-time prediction using a relational DBMS. 
+We assume that you have already run the [a9a binary classification task](https://github.com/myui/hivemall/wiki#a9a-binary-classification).
+
+# Prerequisites
+
+- MySQL
+
+Put mysql-connector-java.jar (JDBC driver) on $SQOOP_HOME/lib.
+
+- [Sqoop](http://sqoop.apache.org/)
+
+Sqoop 1.4.5 does not support Hadoop v2.6.0. So, you need to build packages for Hadoop 2.6.
+To do that you need to edit build.xml and ivy.xml as shown in [this patch](https://gist.github.com/myui/e8db4a31b574103133c6).
+
+# Preparing Model Tables on MySQL
+
+```sql
+create database a9a;
+use a9a;
+
+create user sqoop identified by 'sqoop';
+grant all privileges on a9a.* to 'sqoop'@'%' identified by 'sqoop';
+flush privileges;
+
+create table a9a_model1 (
+  feature int, 
+  weight double
+);
+```
+
+Do not forget to edit bind_address in the MySQL configuration file (/etc/mysql/my.conf) accessible from master and slave nodes of Hadoop.
+
+# Exporting Hive table to MySQL
+
+Check the connectivity to MySQL server using Sqoop.
+
+```sh
+export MYSQL_HOST=dm01
+
+export HADOOP_HOME=/opt/hadoop
+export HADOOP_CONF_DIR=${HADOOP_HOME}/etc/hadoop/
+export HADOOP_COMMON_HOME=${HADOOP_HOME}
+
+bin/sqoop list-tables --connect jdbc:mysql://${MYSQL_HOST}/a9a --username sqoop --password sqoop
+```
+
+Create TSV table because Sqoop cannot directory read Hive tables.
+
+```sql
+create table a9a_model1_tsv 
+  ROW FORMAT DELIMITED 
+    FIELDS TERMINATED BY "\t"
+    LINES TERMINATED BY "\n"
+  STORED AS TEXTFILE
+AS
+select * from a9a_model1;
+```
+
+Check the location of 'a9a_model1_tsv' as follows:
+
+```sql
+desc extended a9a_model1_tsv;
+> location:hdfs://dm01:9000/user/hive/warehouse/a9a.db/a9a_model1_tsv
+```
+
+```sh
+bin/sqoop export \
+--connect jdbc:mysql://${MYSQL_HOST}/a9a \
+--username sqoop --password sqoop \
+--table a9a_model1 \
+--export-dir /user/hive/warehouse/a9a.db/a9a_model1_tsv \
+--input-fields-terminated-by '\t' --input-lines-terminated-by '\n' \
+--batch
+```
+
+When the exporting successfully finishes, you can find entries in the model table in MySQL.
+
+```sql
+mysql> select * from a9a_model1 limit 3;
++---------+---------------------+
+| feature | weight              |
++---------+---------------------+
+|       0 | -0.5761121511459351 |
+|       1 | -1.5259535312652588 |
+|      10 | 0.21053194999694824 |
++---------+---------------------+
+3 rows in set (0.00 sec)
+```
+
+We recommend to create an index of model tables to boost lookups in online prediction.
+
+```sql
+CREATE UNIQUE INDEX a9a_model1_feature_index on a9a_model1 (feature);
+-- USING BTREE;
+```
+
+# Exporting test data from Hadoop to MySQL (optional step)
+
+Prepare a testing data table in Hive which is being exported.
+
+```sql
+create table a9atest_exploded_tsv
+  ROW FORMAT DELIMITED 
+    FIELDS TERMINATED BY "\t"
+    LINES TERMINATED BY "\n"
+  STORED AS TEXTFILE
+AS
+select
+  rowid, 
+  -- label, 
+  extract_feature(feature) as feature,
+  extract_weight(feature) as value
+from
+  a9atest LATERAL VIEW explode(addBias(features)) t AS feature;
+
+desc extended a9atest_exploded_tsv;
+> location:hdfs://dm01:9000/user/hive/warehouse/a9a.db/a9atest_exploded_tsv,
+```
+
+Prepare a test table, importing data from Hadoop.
+
+```sql
+use a9a;
+
+create table a9atest_exploded (
+  rowid bigint,
+  feature int, 
+  value double
+);
+```
+
+Then, run Sqoop to export data from HDFS to MySQL.
+
+```sh
+export MYSQL_HOST=dm01
+
+bin/sqoop export \
+--connect jdbc:mysql://${MYSQL_HOST}/a9a \
+--username sqoop --password sqoop \
+--table a9atest_exploded \
+--export-dir /user/hive/warehouse/a9a.db/a9atest_exploded_tsv \
+--input-fields-terminated-by '\t' --input-lines-terminated-by '\n' \
+--batch
+```
+
+Better to add an index to the rowid column to boost selection by rowids.
+```sql
+CREATE INDEX a9atest_exploded_rowid_index on a9atest_exploded (rowid) USING BTREE;
+```
+
+When the exporting successfully finishes, you can find entries in the test table in MySQL.
+
+```sql
+mysql> select * from a9atest_exploded limit 10;
++-------+---------+-------+
+| rowid | feature | value |
++-------+---------+-------+
+| 12427 |      67 |     1 |
+| 12427 |      73 |     1 |
+| 12427 |      74 |     1 |
+| 12427 |      76 |     1 |
+| 12427 |      82 |     1 |
+| 12427 |      83 |     1 |
+| 12427 |       0 |     1 |
+| 12428 |       5 |     1 |
+| 12428 |       7 |     1 |
+| 12428 |      16 |     1 |
++-------+---------+-------+
+10 rows in set (0.00 sec)
+```
+
+# Online/realtime prediction on MySQL
+
+Define sigmoid function used for a prediction of logistic regression as follows: 
+
+```sql
+DROP FUNCTION IF EXISTS sigmoid;
+DELIMITER $$
+CREATE FUNCTION sigmoid(x DOUBLE)
+  RETURNS DOUBLE
+  LANGUAGE SQL
+BEGIN
+  RETURN 1.0 / (1.0 + EXP(-x));
+END;
+$$
+DELIMITER ;
+```
+
+We assume here that doing prediction for a 'features' having (0,1,10) and each of them is a categorical feature (i.e., the weight is 1.0). Then, you can get the probability by logistic regression simply as follows:
+
+```sql
+select
+  sigmoid(sum(m.weight)) as prob
+from
+  a9a_model1 m
+where
+  m.feature in (0,1,10);
+```
+
+```
++--------------------+
+| prob               |
++--------------------+
+| 0.1310696931351625 |
++--------------------+
+1 row in set (0.00 sec)
+```
+
+Similar to [the way in Hive](https://github.com/myui/hivemall/wiki/a9a-binary-classification-(logistic-regression)#prediction), you can run prediction as follows:
+
+```sql
+select
+  sigmoid(sum(t.value * m.weight)) as prob, 
+  if(sigmoid(sum(t.value * m.weight)) > 0.5, 1.0, 0.0) as predicted
+from
+  a9atest_exploded t LEFT OUTER JOIN
+  a9a_model1 m ON (t.feature = m.feature)
+where
+  t.rowid = 12427; -- prediction on a particular id
+```
+
+Alternatively, you can use SQL views for testing target 't' in the above query.
+
+```
++---------------------+-----------+
+| prob                | predicted |
++---------------------+-----------+
+| 0.05595205126313402 |       0.0 |
++---------------------+-----------+
+1 row in set (0.00 sec)
+```
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/370e2aa3/docs/gitbook/troubleshooting/README.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/troubleshooting/README.md b/docs/gitbook/troubleshooting/README.md
new file mode 100644
index 0000000..e69de29

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/370e2aa3/docs/gitbook/troubleshooting/asterisk.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/troubleshooting/asterisk.md b/docs/gitbook/troubleshooting/asterisk.md
new file mode 100644
index 0000000..49e2f71
--- /dev/null
+++ b/docs/gitbook/troubleshooting/asterisk.md
@@ -0,0 +1,3 @@
+See [HIVE-4181](https://issues.apache.org/jira/browse/HIVE-4181) that asterisk argument without table alias for UDTF is not working. It has been fixed as part of Hive v0.12 release.
+
+A possible workaround is to use asterisk with a table alias, or to specify names of arguments explicitly.
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/370e2aa3/docs/gitbook/troubleshooting/mapjoin_classcastex.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/troubleshooting/mapjoin_classcastex.md b/docs/gitbook/troubleshooting/mapjoin_classcastex.md
new file mode 100644
index 0000000..c48919a
--- /dev/null
+++ b/docs/gitbook/troubleshooting/mapjoin_classcastex.md
@@ -0,0 +1,8 @@
+Map-side join on Tez causes [ClassCastException](http://markmail.org/message/7cwbgupnhah6ggkv) when a serialized table contains array column(s).
+
+[Workaround] Try setting _hive.mapjoin.optimized.hashtable_ off as follows:
+```sql
+set hive.mapjoin.optimized.hashtable=false;
+```
+
+Caution: Fixed in Hive 1.3.0. Refer [HIVE_11051](https://issues.apache.org/jira/browse/HIVE-11051) for the detail.
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/370e2aa3/docs/gitbook/troubleshooting/mapjoin_task_error.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/troubleshooting/mapjoin_task_error.md b/docs/gitbook/troubleshooting/mapjoin_task_error.md
new file mode 100644
index 0000000..02aff2f
--- /dev/null
+++ b/docs/gitbook/troubleshooting/mapjoin_task_error.md
@@ -0,0 +1,8 @@
+From Hive 0.11.0, **hive.auto.convert.join** is [enabled by the default](https://issues.apache.org/jira/browse/HIVE-3297).
+
+When using complex queries using views, the auto conversion sometimes throws SemanticException, cannot serialize object.
+
+Workaround for the exception is to disable **hive.auto.convert.join** before the execution as follows.
+```
+set hive.auto.convert.join=false;
+```
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/370e2aa3/docs/gitbook/troubleshooting/num_mappers.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/troubleshooting/num_mappers.md b/docs/gitbook/troubleshooting/num_mappers.md
new file mode 100644
index 0000000..be01f2a
--- /dev/null
+++ b/docs/gitbook/troubleshooting/num_mappers.md
@@ -0,0 +1,20 @@
+The default _hive.input.format_ is set to _org.apache.hadoop.hive.ql.io.CombineHiveInputFormat_.
+This configuration could give less number of mappers than the split size (i.e., # blocks in HDFS) of the input table.
+
+Try setting _org.apache.hadoop.hive.ql.io.HiveInputFormat_ for _hive.input.format_.
+```
+set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat;
+```
+
+Note Apache Tez uses _org.apache.hadoop.hive.ql.io.HiveInputFormat_ by the default.
+```
+set hive.tez.input.format;
+``` 
+> hive.tez.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat
+
+***
+
+You can then control the maximum number of mappers via setting:
+```
+set mapreduce.job.maps=128;
+```
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/370e2aa3/docs/gitbook/troubleshooting/oom.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/troubleshooting/oom.md b/docs/gitbook/troubleshooting/oom.md
new file mode 100644
index 0000000..643d09a
--- /dev/null
+++ b/docs/gitbook/troubleshooting/oom.md
@@ -0,0 +1,20 @@
+# OOM in mappers
+
+In a certain setting, the default input split size is too large for Hivemall. Due to that, OutOfMemoryError cloud happen on mappers in the middle of training.
+
+Then, revise your a Hadoop setting (**mapred.child.java.opts**/**mapred.map.child.java.opts**) first to use a larger value as possible.
+
+If an OOM error still caused after that, set smaller **mapred.max.split.size** value before training.
+```
+SET mapred.max.split.size=67108864;
+```
+Then, the number of training examples used for each trainer is reduced (as the number of mappers increases) and the trained model would fit in the memory.
+
+# OOM in shuffle/merge
+
+If OOM caused during the merge step, try setting a larger **mapred.reduce.tasks** value before training and revise [shuffle/reduce parameters](http://hadoop.apache.org/docs/r1.0.4/mapred_tutorial.html#Shuffle%2FReduce+Parameters).
+```
+SET mapred.reduce.tasks=64;
+```
+
+If your OOM happened by using amplify(), try using rand_amplify() instead.
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/370e2aa3/src/site/site.xml
----------------------------------------------------------------------
diff --git a/src/site/site.xml b/src/site/site.xml
index fdb8fac..8a20d84 100644
--- a/src/site/site.xml
+++ b/src/site/site.xml
@@ -48,8 +48,8 @@
 				<ribbonOrientation>right</ribbonOrientation>
 				<ribbonColor>red</ribbonColor>
 			</gitHub>			
-            <facebookLike />
-			<twitter>
+            <!-- <facebookLike /> -->
+			<twitter>			    
 				<user>ApacheHivemall</user>
 				<showUser>true</showUser>
 				<showFollowers>false</showFollowers>				
@@ -83,7 +83,7 @@
 		</menu>
 		
 		<menu name="Documentation">
-		  <item name="User Guide" href="/userguide.html" />
+		  <item name="User Guide" href="/userguide/index.html" />
 		  <item name="Overview" href="/overview.html" />
 		  <item name="Wiki" href="https://cwiki.apache.org/confluence/display/HIVEMALL" target="_blank" />
           <item name="FAQ" href="/faq.html" />

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/370e2aa3/src/site/xdoc/index.xml.vm
----------------------------------------------------------------------
diff --git a/src/site/xdoc/index.xml.vm b/src/site/xdoc/index.xml.vm
index 2638458..2dedfca 100644
--- a/src/site/xdoc/index.xml.vm
+++ b/src/site/xdoc/index.xml.vm
@@ -26,6 +26,9 @@
         <script src="js/misc.js" type="text/javascript"/>
     </head>
     <body>
+        <div class="alert alert-info" role="alert">
+            <strong>Info:</strong> We are now in the process of migrating the project repository from <a href="https://github.com/myui/hivemall">Github</a> to <a href="https://github.com/apache/incubator-hivemall">Apache Incubator</a>.
+        </div>
         <div id="carousel-main" class="row">
             <div id="screenshots-carousel" class="carousel slide span10">
                 <!--  Carousel items  -->
@@ -45,9 +48,7 @@
                     <div class="item">
                         <img alt="" src="/images/hivemall_overview_bg.png" height="120px"/>
                         <div class="carousel-caption">
-                            <a href="http://www.slideshare.net/myui/introduction-to-hivemall">
-                            <p>Introduction to Hivemall (slide)</p>
-                            </a>
+                            <p>Introduction to Hivemall <a href="http://www.slideshare.net/myui/introduction-to-hivemall"></a></p>
                         </div>
                     </div>
                 </div>


[03/50] [abbrv] incubator-hivemall git commit: Added a gitbook userguide

Posted by my...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/370e2aa3/docs/gitbook/ft_engineering/scaling.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/ft_engineering/scaling.md b/docs/gitbook/ft_engineering/scaling.md
new file mode 100644
index 0000000..6e7d312
--- /dev/null
+++ b/docs/gitbook/ft_engineering/scaling.md
@@ -0,0 +1,173 @@
+# Min-Max Normalization
+http://en.wikipedia.org/wiki/Feature_scaling#Rescaling
+```sql
+select min(target), max(target)
+from (
+select target from e2006tfidf_train 
+-- union all
+-- select target from e2006tfidf_test 
+) t;
+```
+
+> -7.899578       -0.51940954
+
+```sql
+set hivevar:min_target=-7.899578;
+set hivevar:max_target=-0.51940954;
+
+create or replace view e2006tfidf_train_scaled 
+as
+select 
+  rowid,
+  rescale(target, ${min_target}, ${max_target}) as target, 
+  features
+from 
+  e2006tfidf_train;
+```
+
+# Feature scaling by zscore
+http://en.wikipedia.org/wiki/Standard_score
+
+```sql
+select avg(target), stddev_pop(target)
+from (
+select target from e2006tfidf_train 
+-- union all
+-- select target from e2006tfidf_test 
+) t;
+```
+> -3.566241460963296      0.6278076335455348
+
+```sql
+set hivevar:mean_target=-3.566241460963296;
+set hivevar:stddev_target=0.6278076335455348;
+
+create or replace view e2006tfidf_train_scaled 
+as
+select 
+  rowid,
+  zscore(target, ${mean_target}, ${stddev_target}) as target, 
+  features
+from 
+  e2006tfidf_train;
+```
+
+# Apply Normalization to more complex feature vector
+
+Apply normalization to the following data.
+
+```sql
+select rowid, features from train limit 3;
+```
+
+```
+1       ["weight:69.613","specific_heat:129.07","reflectance:52.111"]
+2       ["weight:70.67","specific_heat:128.161","reflectance:52.446"]
+3       ["weight:72.303","specific_heat:128.45","reflectance:52.853"]
+```
+
+We can create a normalized table as follows:
+
+```sql
+create table train_normalized
+as
+WITH fv as (
+select 
+  rowid, 
+  extract_feature(feature) as feature,
+  extract_weight(feature) as value
+from 
+  train 
+  LATERAL VIEW explode(features) exploded AS feature
+), 
+stats as (
+select
+  feature,
+  -- avg(value) as mean, stddev_pop(value) as stddev
+  min(value) as min, max(value) as max
+from
+  fv
+group by
+  feature
+), 
+norm as (
+select 
+  rowid, 
+  t1.feature, 
+  -- zscore(t1.value, t2.mean, t2.stddev) as zscore
+  rescale(t1.value, t2.min, t2.max) as minmax
+from 
+  fv t1 JOIN
+  stats t2 ON (t1.feature = t2.feature) 
+),
+norm_fv as (
+select
+  rowid, 
+  -- concat(feature, ":", zscore) as feature
+  -- concat(feature, ":", minmax) as feature  -- Before Hivemall v0.3.2-1
+  feature(feature, minmax) as feature         -- Hivemall v0.3.2-1 or later
+from
+  norm
+)
+select 
+  rowid, 
+  collect_list(feature) as features
+from
+  norm_fv
+group by
+  rowid
+;
+```
+
+```
+1       ["reflectance:0.5252967","specific_heat:0.19863537","weight:0.0"]
+2       ["reflectance:0.5950446","specific_heat:0.09166764","weight:0.052084323"]
+3       ["reflectance:0.6797837","specific_heat:0.12567581","weight:0.13255163"]
+...
+```
+
+# Tips for using both min-max and zscore normalization
+
+```sql
+WITH quantative as (
+  select id, true as minmax, "age" as feature, age as value from train
+  union all
+  select id, false as minmax, "balance" as feature, balance as value from train
+  union all
+  select id, true as minmax, "day" as feature, day as value from train
+  union all
+  select id, false as minmax, "duration" as feature, duration as value from train
+  union all
+  select id, false as minmax, "campaign" as feature, campaign as value from train
+  union all
+  select id, false as minmax, "pdays" as feature, if(pdays = -1, 0, pdays) as value from train
+  union all
+  select id, false as minmax,  "previous" as feature, previous as value from train  
+),
+quantative_stats as (
+select
+  feature,
+  avg(value) as mean, stddev_pop(value) as stddev,
+  min(value) as min, max(value) as max
+from
+  quantative
+group by
+  feature
+), 
+quantative_norm as (
+select 
+  t1.id,
+  collect_list(
+   feature(
+      t1.feature, 
+      if(t1.minmax,rescale(t1.value, t2.min, t2.max),zscore(t1.value, t2.mean, t2.stddev))
+    )
+  ) as features
+from 
+  quantative t1
+  JOIN quantative_stats t2 ON (t1.feature = t2.feature)   
+group by
+  t1.id
+)
+...
+```
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/370e2aa3/docs/gitbook/ft_engineering/tfidf.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/ft_engineering/tfidf.md b/docs/gitbook/ft_engineering/tfidf.md
new file mode 100644
index 0000000..e881e10
--- /dev/null
+++ b/docs/gitbook/ft_engineering/tfidf.md
@@ -0,0 +1,149 @@
+This document explains how to compute [TF-IDF](http://en.wikipedia.org/wiki/Tf%E2%80%93idf) with Apache Hive/Hivemall.
+
+What you need to compute TF-IDF is a table/view composing (docid, word) pair, 2 views, and 1 query.
+
+_Note that this feature is supported since Hivemall v0.3-beta3 or later. Macro is supported since Hive 0.12 or later._
+
+# Define macros used in the TF-IDF computation
+```sql
+create temporary macro max2(x INT, y INT)
+if(x>y,x,y);
+
+-- create temporary macro idf(df_t INT, n_docs INT)
+-- (log(10, CAST(n_docs as FLOAT)/max2(1,df_t)) + 1.0);
+
+create temporary macro tfidf(tf FLOAT, df_t INT, n_docs INT)
+tf * (log(10, CAST(n_docs as FLOAT)/max2(1,df_t)) + 1.0);
+```
+
+# Data preparation
+To calculate TF-IDF, you need to prepare a relation consists of (docid,word) tuples.
+```sql
+create external table wikipage (
+  docid int,
+  page string
+)
+ROW FORMAT DELIMITED FIELDS TERMINATED BY '|'
+STORED AS TEXTFILE;
+
+cd ~/tmp
+wget https://gist.githubusercontent.com/myui/190b91a3a792ccfceda0/raw/327acd192da4f96da8276dcdff01b19947a4373c/tfidf_test.tsv
+
+LOAD DATA LOCAL INPATH '/home/myui/tmp/tfidf_test.tsv' INTO TABLE wikipage;
+
+create or replace view wikipage_exploded
+as
+select
+  docid, 
+  word
+from
+  wikipage LATERAL VIEW explode(tokenize(page,true)) t as word
+where
+  not is_stopword(word);
+```
+You can download the data of the wikipage table from [this link]( https://gist.githubusercontent.com/myui/190b91a3a792ccfceda0/raw/327acd192da4f96da8276dcdff01b19947a4373c/tfidf_test.tsv).
+
+# Define views of TF/DF
+```sql
+create or replace view term_frequency 
+as
+select
+  docid, 
+  word,
+  freq
+from (
+select
+  docid,
+  tf(word) as word2freq
+from
+  wikipage_exploded
+group by
+  docid
+) t 
+LATERAL VIEW explode(word2freq) t2 as word, freq;
+
+create or replace view document_frequency
+as
+select
+  word, 
+  count(distinct docid) docs
+from
+  wikipage_exploded
+group by
+  word;
+```
+
+# TF-IDF calculation for each docid/word pair
+```sql
+-- set the total number of documents
+select count(distinct docid) from wikipage;
+set hivevar:n_docs=3;
+
+create or replace view tfidf
+as
+select
+  tf.docid,
+  tf.word, 
+  -- tf.freq * (log(10, CAST(${n_docs} as FLOAT)/max2(1,df.docs)) + 1.0) as tfidf
+  tfidf(tf.freq, df.docs, ${n_docs}) as tfidf
+from
+  term_frequency tf 
+  JOIN document_frequency df ON (tf.word = df.word)
+order by 
+  tfidf desc;
+```
+
+The result will be as follows:
+```
+docid  word     tfidf
+1       justice 0.1641245850805637
+3       knowledge       0.09484606645205085
+2       action  0.07033910867777095
+1       law     0.06564983513276658
+1       found   0.06564983513276658
+1       religion        0.06564983513276658
+1       discussion      0.06564983513276658
+  ...
+  ...
+2       act     0.017584777169442737
+2       virtues 0.017584777169442737
+2       well    0.017584777169442737
+2       willingness     0.017584777169442737
+2       find    0.017584777169442737
+2       1       0.014001086678120098
+2       experience      0.014001086678120098
+2       often   0.014001086678120098
+```
+The above result is considered to be appropriate as docid 1, 2, and 3 are the Wikipedia entries of Justice, Wisdom, and Knowledge, respectively.
+
+# Feature Vector with TF-IDF values
+
+```sql
+select
+  docid, 
+  -- collect_list(concat(word, ":", tfidf)) as features -- Hive 0.13 or later
+  collect_list(feature(word, tfidf)) as features -- Hivemall v0.3.4 & Hive 0.13 or later
+  -- collect_all(concat(word, ":", tfidf)) as features -- before Hive 0.13
+from 
+  tfidf
+group by
+  docid;
+```
+
+```
+1       ["justice:0.1641245850805637","found:0.06564983513276658","discussion:0.06564983513276658","law:0.065
+64983513276658","based:0.06564983513276658","religion:0.06564983513276658","viewpoints:0.03282491756638329","
+rationality:0.03282491756638329","including:0.03282491756638329","context:0.03282491756638329","concept:0.032
+82491756638329","rightness:0.03282491756638329","general:0.03282491756638329","many:0.03282491756638329","dif
+fering:0.03282491756638329","fairness:0.03282491756638329","social:0.03282491756638329","broadest:0.032824917
+56638329","equity:0.03282491756638329","includes:0.03282491756638329","theology:0.03282491756638329","ethics:
+0.03282491756638329","moral:0.03282491756638329","numerous:0.03282491756638329","philosophical:0.032824917566
+38329","application:0.03282491756638329","perspectives:0.03282491756638329","procedural:0.03282491756638329",
+"realm:0.03282491756638329","divided:0.03282491756638329","concepts:0.03282491756638329","attainment:0.032824
+91756638329","fields:0.03282491756638329","often:0.026135361945200226","philosophy:0.026135361945200226","stu
+dy:0.026135361945200226"]
+2       ["action:0.07033910867777095","wisdom:0.05275433288400458","one:0.05275433288400458","understanding:0
+.04200326112968063","judgement:0.035169554338885474","apply:0.035169554338885474","disposition:0.035169554338
+885474","given:0.035169554338885474"
+...
+```
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/370e2aa3/docs/gitbook/ft_engineering/vectorizer.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/ft_engineering/vectorizer.md b/docs/gitbook/ft_engineering/vectorizer.md
new file mode 100644
index 0000000..bc929a5
--- /dev/null
+++ b/docs/gitbook/ft_engineering/vectorizer.md
@@ -0,0 +1,42 @@
+## Feature Vectorizer
+
+`array<string> vectorize_feature(array<string> featureNames, ...)` is useful to generate a feature vector for each row, from a table.
+
+```sql
+select vectorize_features(array("a","b"),"0.2","0.3") from dual;
+>["a:0.2","b:0.3"]
+
+-- avoid zero weight
+select vectorize_features(array("a","b"),"0.2",0) from dual;
+> ["a:0.2"]
+
+-- true boolean value is treated as 1.0 (a categorical value w/ its column name)
+select vectorize_features(array("a","b","bool"),0.2,0.3,true) from dual;
+> ["a:0.2","b:0.3","bool:1.0"]
+
+-- an example to generate feature vectors from table
+select * from dual;
+> 1                                         
+select vectorize_features(array("a"),*) from dual;
+> ["a:1.0"]
+
+-- has categorical feature
+select vectorize_features(array("a","b","wheather"),"0.2","0.3","sunny") from dual;
+> ["a:0.2","b:0.3","whether#sunny"]
+```
+
+```sql
+select
+  id,
+  vectorize_features(
+    array("age","job","marital","education","default","balance","housing","loan","contact","day","month","duration","campaign","pdays","previous","poutcome"), 
+    age,job,marital,education,default,balance,housing,loan,contact,day,month,duration,campaign,pdays,previous,poutcome
+  ) as features,
+  y
+from
+  train
+limit 2;
+
+> 1       ["age:39.0","job#blue-collar","marital#married","education#secondary","default#no","balance:1756.0","housing#yes","loan#no","contact#cellular","day:3.0","month#apr","duration:939.0","campaign:1.0","pdays:-1.0","poutcome#unknown"]   1
+> 2       ["age:51.0","job#entrepreneur","marital#married","education#primary","default#no","balance:1443.0","housing#no","loan#no","contact#cellular","day:18.0","month#feb","duration:172.0","campaign:10.0","pdays:-1.0","poutcome#unknown"]   1
+```
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/370e2aa3/docs/gitbook/getting_started/README.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/getting_started/README.md b/docs/gitbook/getting_started/README.md
new file mode 100644
index 0000000..27870e5
--- /dev/null
+++ b/docs/gitbook/getting_started/README.md
@@ -0,0 +1 @@
+# Summary
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/370e2aa3/docs/gitbook/getting_started/input-format.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/getting_started/input-format.md b/docs/gitbook/getting_started/input-format.md
new file mode 100644
index 0000000..272d3eb
--- /dev/null
+++ b/docs/gitbook/getting_started/input-format.md
@@ -0,0 +1,214 @@
+This page explains the input format of training data in Hivemall. 
+Here, we use [EBNF](http://en.wikipedia.org/wiki/Extended_Backus%E2%80%93Naur_Form)-like notation for describing the format.
+
+<!-- toc -->
+
+# Input Format for Classification 
+
+The classifiers of Hivemall takes 2 (or 3) arguments: *features*, *label*, and *options* (a.k.a. [hyperparameters](http://en.wikipedia.org/wiki/Hyperparameter)). The first two arguments of training functions (e.g., [logress](https://github.com/myui/hivemall/wiki/a9a-binary-classification-(logistic-regression)) and [train_scw](https://github.com/myui/hivemall/wiki/news20-binary-classification-%232-(CW,-AROW,-SCW))) represents training examples. 
+
+In Statistics, *features* and *label* are called [Explanatory variable and Response Variable](http://www.oswego.edu/~srp/stats/variable_types.htm), respectively.
+
+# Features format (for classification and regression)
+
+The format of *features* is common between (binary and multi-class) classification and regression.
+Hivemall accepts ARRAY&lt;INT|BIGINT|TEXT> for the type of *features* column.
+
+Hivemall uses a *sparse* data format (cf. [Compressed Row Storage](http://netlib.org/linalg/html_templates/node91.html)) which is similar to [LIBSVM](http://stackoverflow.com/questions/12112558/read-write-data-in-libsvm-format) and [Vowpal Wabbit](https://github.com/JohnLangford/vowpal_wabbit/wiki/Input-format).
+
+The format of each feature in an array is as follows:
+```
+feature ::= <index>:<weight> or <index>
+```
+
+Each element of *index* or *weight* then accepts the following format:
+```
+index ::= <INT | BIGINT | TEXT>
+weight ::= <FLOAT>
+```
+
+The *index* are usually a number (INT or BIGINT) starting from 1. 
+Here is an instance of a features.
+```
+10:3.4  123:0.5  34567:0.231
+```
+
+*Note:* As mentioned later, *index* "0" is reserved for a [Bias/Dummy variable](https://github.com/myui/hivemall/wiki/Using-explicit-addBias()-for-a-better-prediction).
+
+In addition to numbers, you can use a TEXT value for an index. For example, you can use array("height:1.5", "length:2.0") for the features.
+```
+"height:1.5" "length:2.0"
+```
+
+## Quantitative and Categorical variables
+
+A [quantitative variable](http://www.oswego.edu/~srp/stats/variable_types.htm) must have an *index* entry.
+
+Hivemall (v0.3.1 or later) provides *add_feature_index* function which is useful for adding indexes to quantitative variables. 
+
+```sql
+select add_feature_index(array(3,4.0,5)) from dual;
+```
+> ["1:3.0","2:4.0","3:5.0"]
+
+You can omit specifying *weight* for each feature e.g. for [Categorical variables](http://www.oswego.edu/~srp/stats/variable_types.htm) as follows:
+```
+feature ::= <index>
+```
+Note 1.0 is used for the weight when omitting *weight*. 
+
+## Bias/Dummy Variable in features
+
+Note that "0" is reserved for a Bias variable (called dummy variable in Statistics). 
+
+The [addBias](https://github.com/myui/hivemall/wiki/Using-explicit-addBias()-for-a-better-prediction) function is Hivemall appends "0:1.0" as an element of array in *features*.
+
+## Feature hashing
+
+Hivemall supports [feature hashing/hashing trick](http://en.wikipedia.org/wiki/Feature_hashing) through [mhash function](https://github.com/myui/hivemall/wiki/KDDCup-2012-track-2-CTR-prediction-dataset#converting-feature-representation-by-feature-hashing).
+
+The mhash function takes a feature (i.e., *index*) of TEXT format and generates a hash number of a range from 1 to 2^24 (=16777216) by the default setting.
+
+Feature hashing is useful where the dimension of feature vector (i.e., the number of elements in *features*) is so large. Consider applying [mhash function]((https://github.com/myui/hivemall/wiki/KDDCup-2012-track-2-CTR-prediction-dataset#converting-feature-representation-by-feature-hashing)) when a prediction model does not fit in memory and OutOfMemory exception happens.
+
+In general, you don't need to use mhash when the dimension of feature vector is less than 16777216.
+If feature *index* is very long TEXT (e.g., "xxxxxxx-yyyyyy-weight:55.3") and uses huge memory spaces, consider using mhash as follows:
+```sql
+-- feature is v0.3.2 or before
+concat(mhash(extract_feature("xxxxxxx-yyyyyy-weight:55.3")), ":", extract_weight("xxxxxxx-yyyyyy-weight:55.3"))
+
+-- feature is v0.3.2-1 or later
+feature(mhash(extract_feature("xxxxxxx-yyyyyy-weight:55.3")), extract_weight("xxxxxxx-yyyyyy-weight:55.3"))
+```
+> 43352:55.3
+
+## Feature Normalization
+
+Feature (weight) normalization is important in machine learning. Please refer [https://github.com/myui/hivemall/wiki/Feature-scaling](https://github.com/myui/hivemall/wiki/Feature-scaling) for detail.
+
+***
+
+# Label format in Binary Classification
+
+The *label* must be an *INT* typed column and the values are positive (+1) or negative (-1) as follows:
+```
+<label> ::= 1 | -1
+```
+
+Alternatively, you can use the following format that represents 1 for a positive example and 0 for a negative example: 
+```
+<label> ::= 0 | 1
+```
+
+# Label format in Multi-class Classification
+
+You can used any PRIMITIVE type in the multi-class *label*.  
+
+```
+<label> ::= <primitive type>
+```
+
+Typically, the type of label column will be INT, BIGINT, or TEXT.
+
+***
+
+# Input format in Regression
+
+In regression, response/predictor variable (we denote it as *target*) is a real number.
+
+Before Hivemall v0.3, we accepts only FLOAT type for *target*.
+```
+<target> ::= <FLOAT> 
+```
+
+You need to explicitly cast a double value of *target* to float as follows:
+```sql
+CAST(target as FLOAT) 
+```
+
+On the other hand, Hivemall v0.3 or later accepts double compatible numbers in *target*.
+```
+<target> ::= <FLOAT | DOUBLE | INT | TINYINT | SMALLINT| BIGINT > 
+```
+
+## Target in Logistic Regression
+
+Logistic regression is actually a binary classification scheme while it can produce probabilities of positive of a training example. 
+
+A *target* value of a training input must be in range 0.0 to 1.0, specifically 0.0 or 1.0.
+
+***
+
+# Helper functions
+
+```sql
+-- hivemall v0.3.2 and before
+select concat("weight",":",55.0);
+
+-- hivemall v0.3.2-1 and later
+select feature("weight", 55.0);
+```
+> weight:55.0
+
+```sql
+select extract_feature("weight:55.0"), extract_weight("weight:55.0");
+```
+> weight | 55.0
+
+```sql
+-- hivemall v0.4.0 and later
+select feature_index(array("10:0.2","7:0.3","9"));
+```
+> [10,7,9]
+
+```sql
+select 
+  convert_label(-1), convert_label(1), convert_label(0.0f), convert_label(1.0f)
+from 
+  dual;
+```
+> 0.0f | 1.0f | -1 | 1
+
+## Quantitative Features
+
+`array<string> quantitative_features(array<string> featureNames, ...)` is a helper function to create sparse quantitative features from a table.
+
+```sql
+select quantitative_features(array("apple","value"),1,120.3);
+```
+> ["apple:1.0","value:120.3"]
+
+## Categorical Features
+
+`array<string> categorical_features(array<string> featureNames, ...)` is a helper function to create sparse categorical features from a table.
+
+```sql
+select categorical_features(
+  array("is_cat","is_dog","is_lion","is_pengin","species"),
+  1, 0, 1.0, true, "dog"
+);
+```
+> ["is_cat#1","is_dog#0","is_lion#1.0","is_pengin#true","species#dog"]
+
+## Preparing training data table 
+
+You can create a training data table as follows:
+
+```sql
+select 
+  rowid() as rowid,
+  concat_array(
+    array("bias:1.0"),
+    categorical_features( 
+      array("id", "name"),
+      id, name
+    ),
+    quantitative_features(
+      array("height", "weight"),
+      height, weight
+    )
+  ) as features, 
+  click_or_not as label
+from
+  table;
+```
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/370e2aa3/docs/gitbook/getting_started/installation.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/getting_started/installation.md b/docs/gitbook/getting_started/installation.md
new file mode 100644
index 0000000..bb1920e
--- /dev/null
+++ b/docs/gitbook/getting_started/installation.md
@@ -0,0 +1,25 @@
+Prerequisites
+============
+
+* Hive v0.12 or later
+* Java 7 or later
+* [hivemall-core-xxx-with-dependencies.jar](https://github.com/myui/hivemall/releases)
+* [define-all.hive](https://github.com/myui/hivemall/releases)
+
+Installation
+============
+
+Add the following two lines to your `$HOME/.hiverc` file.
+
+```
+add jar /home/myui/tmp/hivemall-core-xxx-with-dependencies.jar;
+source /home/myui/tmp/define-all.hive;
+```
+
+This automatically loads all Hivemall functions every time you start a Hive session. Alternatively, you can run the following command each time.
+
+```
+$ hive
+add jar /tmp/hivemall-core-xxx-with-dependencies.jar;
+source /tmp/define-all.hive;
+```
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/370e2aa3/docs/gitbook/getting_started/permanent-functions.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/getting_started/permanent-functions.md b/docs/gitbook/getting_started/permanent-functions.md
new file mode 100644
index 0000000..aab399b
--- /dev/null
+++ b/docs/gitbook/getting_started/permanent-functions.md
@@ -0,0 +1,42 @@
+Hive v0.13 or later supports [permanent functions](https://cwiki.apache.org/confluence/display/Hive/LanguageManual+DDL#LanguageManualDDL-Create/DropFunction) that live across sessions.
+
+Permanent functions are useful when you are using Hive through Hiveserver or to avoid hivemall installation for each session.
+
+_Note: This feature is supported since hivemall-0.3 beta 3 or later._
+
+<!-- toc -->
+
+# Put hivemall jar to HDFS
+
+First, put hivemall jar to HDFS as follows:
+```sh
+hadoop fs -mkdir -p /apps/hivemall
+hadoop fs -put hivemall-with-dependencies.jar /apps/hivemall
+```
+
+# Create permanent functions
+
+_The following is an auxiliary step to define functions for hivemall databases, not for the default database._
+```sql
+CREATE DATABASE IF NOT EXISTS hivemall;
+USE hivemall;
+```
+
+Then, create permanent functions using [define-all-as-permanent.hive](https://github.com/myui/hivemall/blob/master/resources/ddl/define-all-as-permanent.hive), a DDL script to define permanent UDFs.
+```sql
+set hivevar:hivemall_jar=hdfs:///apps/hivemall/hivemall-with-dependencies.jar;
+
+source /tmp/define-all-as-permanent.hive;
+```
+
+# Confirm functions
+
+```sql
+show functions "hivemall.*";
+
+> hivemall.adadelta
+> hivemall.adagrad
+```
+
+> #### Caution
+You need to specify "hivemall." prefix to call hivemall UDFs in your queries if UDFs are loaded into non-default scheme, in this case _hivemall_.
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/370e2aa3/docs/gitbook/misc/generic_funcs.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/misc/generic_funcs.md b/docs/gitbook/misc/generic_funcs.md
new file mode 100644
index 0000000..1769699
--- /dev/null
+++ b/docs/gitbook/misc/generic_funcs.md
@@ -0,0 +1,210 @@
+This page describes a list of useful Hivemall generic functions.
+
+# Array functions
+
+## Array UDFs
+
+- `array_concat(array<ANY> x1, array<ANY> x2, ..)` - Returns a concatenated array
+
+```sql
+select array_concat(array(1),array(2,3));
+> [1,2,3]
+```
+
+- `array_intersect(array<ANY> x1, array<ANY> x2, ..)` - Returns an intersect of given arrays
+
+```sql
+select array_intersect(array(1,3,4),array(2,3,4),array(3,5));
+> [3]
+```
+
+- `array_remove(array<int|text> original, int|text|array<int> target)` - Returns an array that the target is removed from the original array
+
+```sql
+select array_remove(array(1,null,3),array(null));
+> [3]
+
+select array_remove(array("aaa","bbb"),"bbb");
+> ["aaa"]
+```
+
+- `sort_and_uniq_array(array<int>)` - Takes an array of type int and returns a sorted array in a natural order with duplicate elements eliminated
+
+```sql
+select sort_and_uniq_array(array(3,1,1,-2,10));
+> [-2,1,3,10]
+```
+
+- `subarray_endwith(array<int|text> original, int|text key)` - Returns an array that ends with the specified key
+
+```sql
+select subarray_endwith(array(1,2,3,4), 3);
+> [1,2,3]
+```
+
+- `subarray_startwith(array<int|text> original, int|text key)` - Returns an array that starts with the specified key
+
+```sql
+select subarray_startwith(array(1,2,3,4), 2);
+> [2,3,4]
+```
+
+- `subarray(array<int> orignal, int fromIndex, int toIndex)` - Returns a slice of the original array between the inclusive fromIndex and the exclusive toIndex
+
+```sql
+select subarray(array(1,2,3,4,5,6), 2,4);
+> [3,4]
+```
+
+## Array UDAFs
+
+- `array_avg(array<NUMBER>)` - Returns an array<double> in which each element is the mean of a set of numbers
+
+- `array_sum(array<NUMBER>)` - Returns an array<double> in which each element is summed up
+
+# Bitset functions
+
+## Bitset UDF
+
+- `to_bits(int[] indexes)` - Returns an bitset representation if the given indexes in long[]
+
+```sql
+select to_bits(array(1,2,3,128));
+>[14,-9223372036854775808]
+```
+
+- `unbits(long[] bitset)` - Returns an long array of the give bitset representation
+
+```sql
+select unbits(to_bits(array(1,4,2,3)));
+> [1,2,3,4]
+```
+
+- `bits_or(array<long> b1, array<long> b2, ..)` - Returns a logical OR given bitsets
+
+```sql
+select unbits(bits_or(to_bits(array(1,4)),to_bits(array(2,3))));
+> [1,2,3,4]
+```
+
+## Bitset UDAF
+
+- `bits_collect(int|long x)` - Returns a bitset in array<long>
+
+
+# Compression functions
+
+- `deflate(TEXT data [, const int compressionLevel])` - Returns a compressed BINARY obeject by using Deflater.
+The compression level must be in range [-1,9]
+
+```sql
+select base91(deflate('aaaaaaaaaaaaaaaabbbbccc'));
+> AA+=kaIM|WTt!+wbGAA
+```
+
+- `inflate(BINARY compressedData)` - Returns a decompressed STRING by using Inflater
+
+
+```sql
+select inflate(unbase91(base91(deflate('aaaaaaaaaaaaaaaabbbbccc'))));
+> aaaaaaaaaaaaaaaabbbbccc
+```
+
+# Map functions
+
+## Map UDFs
+
+- `map_get_sum(map<int,float> src, array<int> keys)` - Returns sum of values that are retrieved by keys
+
+- `map_tail_n(map SRC, int N)` - Returns the last N elements from a sorted array of SRC
+
+## MAP UDAFs
+
+- `to_map(key, value)` - Convert two aggregated columns into a key-value map
+
+- `to_ordered_map(key, value [, const boolean reverseOrder=false])` - Convert two aggregated columns into an ordered key-value map
+
+
+# MapReduce functions
+
+- `rowid()` - Returns a generated row id of a form {TASK_ID}-{SEQUENCE_NUMBER}
+
+- `taskid()` - Returns the value of mapred.task.partition
+
+# Math functions
+
+- `sigmoid(x)` - Returns 1.0 / (1.0 + exp(-x))
+
+# Text processing functions
+
+- `base91(binary)` - Convert the argument from binary to a BASE91 string
+
+```sql
+select base91(deflate('aaaaaaaaaaaaaaaabbbbccc'));
+> AA+=kaIM|WTt!+wbGAA
+```
+
+- `unbase91(string)` - Convert a BASE91 string to a binary
+
+```sql
+select inflate(unbase91(base91(deflate('aaaaaaaaaaaaaaaabbbbccc'))));
+> aaaaaaaaaaaaaaaabbbbccc
+```
+
+- `normalize_unicode(string str [, string form])` - Transforms `str` with the specified normalization form. The `form` takes one of NFC (default), NFD, NFKC, or NFKD
+
+```sql
+select normalize_unicode('\uff8a\uff9d\uff76\uff78\uff76\uff85','NFKC');
+> \u30cf\u30f3\u30ab\u30af\u30ab\u30ca
+
+select normalize_unicode('\u3231\u3327\u3326\u2162','NFKC');
+> (\u682a)\u30c8\u30f3\u30c9\u30ebIII
+```
+
+- `split_words(string query [, string regex])` - Returns an array<text> containing splitted strings
+
+- `is_stopword(string word)` - Returns whether English stopword or not
+
+- `tokenize(string englishText [, boolean toLowerCase])` - Returns words in array<string>
+
+- `tokenize_ja(String line [, const string mode = "normal", const list<string> stopWords, const list<string> stopTags])` - returns tokenized strings in array<string>
+
+```sql
+select tokenize_ja("kuromoji\u3092\u4f7f\u3063\u305f\u5206\u304b\u3061\u66f8\u304d\u306e\u30c6\u30b9\u30c8\u3067\u3059\u3002\u7b2c\u4e8c\u5f15\u6570\u306b\u306fnormal/search/extended\u3092\u6307\u5b9a\u3067\u304d\u307e\u3059\u3002\u30c7\u30d5\u30a9\u30eb\u30c8\u3067\u306fnormal\u30e2\u30fc\u30c9\u3067\u3059\u3002");
+
+> ["kuromoji","\u4f7f\u3046","\u5206\u304b\u3061\u66f8\u304d","\u30c6\u30b9\u30c8","\u7b2c","\u4e8c","\u5f15\u6570","normal","search","extended","\u6307\u5b9a","\u30c7\u30d5\u30a9\u30eb\u30c8","normal"," \u30e2\u30fc\u30c9"]
+```
+
+https://github.com/myui/hivemall/wiki/Tokenizer
+
+# Other functions
+
+- `convert_label(const int|const float)` - Convert from -1|1 to 0.0f|1.0f, or from 0.0f|1.0f to -1|1
+
+- `each_top_k(int K, Object group, double cmpKey, *)` - Returns top-K values (or tail-K values when k is less than 0)
+
+https://github.com/myui/hivemall/wiki/Efficient-Top-k-computation-on-Apache-Hive-using-Hivemall-UDTF
+
+- `generate_series(const int|bigint start, const int|bigint end)` - Generate a series of values, from start to end
+
+```sql
+WITH dual as (
+  select 1
+)
+select generate_series(1,9)
+from dual;
+
+1
+2
+3
+4
+5
+6
+7
+8
+9
+```
+
+A similar function to PostgreSQL's `generate_serics`.
+http://www.postgresql.org/docs/current/static/functions-srf.html
+- `x_rank(KEY)` - Generates a pseudo sequence number starting from 1 for each key
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/370e2aa3/docs/gitbook/misc/tokenizer.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/misc/tokenizer.md b/docs/gitbook/misc/tokenizer.md
new file mode 100644
index 0000000..cd2ce08
--- /dev/null
+++ b/docs/gitbook/misc/tokenizer.md
@@ -0,0 +1,30 @@
+# Tokenizer for English Texts
+
+Hivemall provides simple English text tokenizer UDF that has following syntax:
+```sql
+tokenize(text input, optional boolean toLowerCase = false)
+```
+
+# Tokenizer for Japanese Texts
+
+Hivemall-NLP module provides a Japanese text tokenizer UDF using [Kuromoji](https://github.com/atilika/kuromoji). 
+
+First of all, you need to issue the following DDLs to use the NLP module. Note NLP module is not included in [hivemall-with-dependencies.jar](https://github.com/myui/hivemall/releases).
+
+> add jar /tmp/[hivemall-nlp-xxx-with-dependencies.jar](https://github.com/myui/hivemall/releases);
+
+> source /tmp/[define-additional.hive](https://github.com/myui/hivemall/releases);
+
+The signature of the UDF is as follows:
+```sql
+tokenize_ja(text input, optional const text mode = "normal", optional const array<string> stopWords, optional const array<string> stopTags)
+```
+_Caution: `tokenize_ja` is supported since Hivemall v0.4.1 and later._
+
+It's basic usage is as follows:
+```sql
+select tokenize_ja("kuromoji\u3092\u4f7f\u3063\u305f\u5206\u304b\u3061\u66f8\u304d\u306e\u30c6\u30b9\u30c8\u3067\u3059\u3002\u7b2c\u4e8c\u5f15\u6570\u306b\u306fnormal/search/extended\u3092\u6307\u5b9a\u3067\u304d\u307e\u3059\u3002\u30c7\u30d5\u30a9\u30eb\u30c8\u3067\u306fnormal\u30e2\u30fc\u30c9\u3067\u3059\u3002");
+```
+> ["kuromoji","\u4f7f\u3046","\u5206\u304b\u3061\u66f8\u304d","\u30c6\u30b9\u30c8","\u7b2c","\u4e8c","\u5f15\u6570","normal","search","extended","\u6307\u5b9a","\u30c7\u30d5\u30a9\u30eb\u30c8","normal","\u30e2\u30fc\u30c9"]
+
+For detailed APIs, please refer Javadoc of [JapaneseAnalyzer](https://lucene.apache.org/core/5_3_1/analyzers-kuromoji/org/apache/lucene/analysis/ja/JapaneseAnalyzer.html) as well.
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/370e2aa3/docs/gitbook/misc/topk.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/misc/topk.md b/docs/gitbook/misc/topk.md
new file mode 100644
index 0000000..dcd545a
--- /dev/null
+++ b/docs/gitbook/misc/topk.md
@@ -0,0 +1,288 @@
+`each_top_k(int k, ANY group, double value, arg1, arg2, ..., argN)` returns a top-k records for each `group`. It returns a relation consists of `(int rank, double value, arg1, arg2, .., argN)`.
+
+This function is particularly useful for applying a similarity/distance function where the computation complexity is **O(nm)**.
+
+`each_top_k` is very fast when compared to other methods running top-k queries (e.g., [`rank/distributed by`](https://ragrawal.wordpress.com/2011/11/18/extract-top-n-records-in-each-group-in-hadoophive/)) in Hive.
+
+## Caution
+* `each_top_k` is supported from Hivemall v0.3.2-3 or later.
+* This UDTF assumes that input records are sorted by `group`. Use `DISTRIBUTED BY group SORTED BY group` to ensure that. Or, you can use `LEFT OUTER JOIN` for certain cases.
+* It takes variable lengths arguments in `argN`. 
+* The third argument `value` is used for the comparison.
+* `Any number types` or `timestamp` are accepted for the type of `value`.
+* If k is less than 0, reverse order is used and `tail-K` records are returned for each `group`.
+* Note that this function returns [a pseudo ranking](http://www.michaelpollmeier.com/selecting-top-k-items-from-a-list-efficiently-in-java-groovy/) for top-k. It always returns `at-most K` records for each group. The ranking scheme is similar to `dense_rank` but slightly different in certain cases.
+
+# Usage
+
+## top-k clicks 
+
+http://stackoverflow.com/questions/9390698/hive-getting-top-n-records-in-group-by-query/32559050#32559050
+
+```sql
+set hivevar:k=5;
+
+select
+  page-id, 
+  user-id,
+  clicks
+from (
+  select
+    each_top_k(${k}, page-id, clicks, page-id, user-id)
+      as (rank, clicks, page-id, user-id)
+  from (
+    select
+      page-id, user-id, clicks
+    from
+      mytable
+    DISTRIBUTE BY page-id SORT BY page-id
+  ) t1
+) t2
+order by page-id ASC, clicks DESC;
+```
+
+## Top-k similarity computation
+
+```sql
+set hivevar:k=10;
+
+SELECT
+  each_top_k(
+    ${k}, t2.id, angular_similarity(t2.features, t1.features), 
+    t2.id, 
+    t1.id,  
+    t1.y
+  ) as (rank, similarity, base_id, neighbor_id, y)
+FROM
+  test_hivemall t2 
+  LEFT OUTER JOIN train_hivemall t1;
+```
+
+```
+1       0.8594650626182556      12      10514   0
+2       0.8585299849510193      12      11719   0
+3       0.856602132320404       12      21009   0
+4       0.8562054634094238      12      17582   0
+5       0.8516314029693604      12      22006   0
+6       0.8499397039413452      12      25364   0
+7       0.8467264771461487      12      900     0
+8       0.8463355302810669      12      8018    0
+9       0.8439178466796875      12      7041    0
+10      0.8438876867294312      12      21595   0
+1       0.8390793800354004      25      21125   0
+2       0.8344510793685913      25      14073   0
+3       0.8340602517127991      25      9008    0
+4       0.8328862190246582      25      6598    0
+5       0.8301891088485718      25      943     0
+6       0.8271955251693726      25      20400   0
+7       0.8255619406700134      25      10922   0
+8       0.8241575956344604      25      8477    0
+9       0.822281539440155       25      25977   0
+10      0.8205751180648804      25      21115   0
+1       0.9761330485343933      34      2513    0
+2       0.9536819458007812      34      8697    0
+3       0.9531533122062683      34      7326    0
+4       0.9493276476860046      34      15173   0
+5       0.9480557441711426      34      19468   0
+...
+```
+
+### Explicit grouping using `distribute by` and `sort by`
+
+```sql
+SELECT
+  each_top_k(
+    10, id1, angular_similarity(features1, features2), 
+    id1, 
+    id2,  
+    y
+  ) as (rank, similarity, id, other_id, y)
+FROM (
+select
+  t1.id as id1,
+  t2.id as id2,
+  t1.features as features1,
+  t2.features as features2,
+  t1.y
+from
+  train_hivemall t1
+  CROSS JOIN test_hivemall t2
+DISTRIBUTE BY id1 SORT BY id1
+) t;
+```
+
+### Parallelization of similarity computation using WITH clause
+
+```sql
+create table similarities
+as
+WITH test_rnd as (
+select
+  rand(31) as rnd,
+  id,
+  features
+from
+  test_hivemall
+),
+t01 as (
+select
+ id,
+ features
+from
+ test_rnd
+where
+ rnd < 0.2
+),
+t02 as (
+select
+ id,
+ features
+from
+ test_rnd
+where
+ rnd >= 0.2 and rnd < 0.4
+),
+t03 as (
+select
+ id,
+ features
+from
+ test_rnd
+where
+ rnd >= 0.4 and rnd < 0.6
+),
+t04 as (
+select
+ id,
+ features
+from
+ test_rnd
+where
+ rnd >= 0.6 and rnd < 0.8
+),
+t05 as (
+select
+ id,
+ features
+from
+ test_rnd
+where
+ rnd >= 0.8
+),
+s01 as (
+SELECT
+  each_top_k(
+    10, t2.id, angular_similarity(t2.features, t1.features), 
+    t2.id, 
+    t1.id,  
+    t1.y
+  ) as (rank, similarity, base_id, neighbor_id, y)
+FROM
+  t01 t2 
+  LEFT OUTER JOIN train_hivemall t1
+),
+s02 as (
+SELECT
+  each_top_k(
+    10, t2.id, angular_similarity(t2.features, t1.features), 
+    t2.id, 
+    t1.id,  
+    t1.y
+  ) as (rank, similarity, base_id, neighbor_id, y)
+FROM
+  t02 t2 
+  LEFT OUTER JOIN train_hivemall t1
+),
+s03 as (
+SELECT
+  each_top_k(
+    10, t2.id, angular_similarity(t2.features, t1.features), 
+    t2.id, 
+    t1.id,  
+    t1.y
+  ) as (rank, similarity, base_id, neighbor_id, y)
+FROM
+  t03 t2 
+  LEFT OUTER JOIN train_hivemall t1
+),
+s04 as (
+SELECT
+  each_top_k(
+    10, t2.id, angular_similarity(t2.features, t1.features), 
+    t2.id, 
+    t1.id,  
+    t1.y
+  ) as (rank, similarity, base_id, neighbor_id, y)
+FROM
+  t04 t2 
+  LEFT OUTER JOIN train_hivemall t1
+),
+s05 as (
+SELECT
+  each_top_k(
+    10, t2.id, angular_similarity(t2.features, t1.features), 
+    t2.id, 
+    t1.id,  
+    t1.y
+  ) as (rank, similarity, base_id, neighbor_id, y)
+FROM
+  t05 t2 
+  LEFT OUTER JOIN train_hivemall t1
+)
+select * from s01
+union all
+select * from s02
+union all
+select * from s03
+union all
+select * from s04
+union all
+select * from s05;
+```
+
+## tail-K
+
+```sql
+set hivevar:k=-10;
+
+SELECT
+  each_top_k(
+    ${k}, t2.id, angular_similarity(t2.features, t1.features), 
+    t2.id, 
+    t1.id,  
+    t1.y
+  ) as (rank, similarity, base_id, neighbor_id, y)
+FROM
+  test_hivemall t2 
+  LEFT OUTER JOIN train_hivemall t1
+-- limit 25
+```
+
+```
+1       0.4383084177970886      1       7503    0
+2       0.44166821241378784     1       10143   0
+3       0.4424300789833069      1       11073   0
+4       0.44254064559936523     1       17782   0
+5       0.4442034363746643      1       18556   0
+6       0.45163780450820923     1       3786    0
+7       0.45244503021240234     1       10242   0
+8       0.4525672197341919      1       21657   0
+9       0.4527127146720886      1       17218   0
+10      0.45314133167266846     1       25141   0
+1       0.44030147790908813     2       3786    0
+2       0.4408798813819885      2       23386   0
+3       0.44112563133239746     2       11073   0
+4       0.4415401816368103      2       22853   0
+5       0.4422193765640259      2       21657   0
+6       0.4429032802581787      2       10143   0
+7       0.4435907006263733      2       24413   0
+8       0.44569307565689087     2       7503    0
+9       0.4460843801498413      2       25141   0
+10      0.4464914798736572      2       24289   0
+1       0.43862903118133545     3       23150   1
+2       0.4398220181465149      3       9881    1
+3       0.44283604621887207     3       27121   0
+4       0.4432108402252197      3       26220   1
+5       0.44323229789733887     3       18541   0
+...
+```
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/370e2aa3/docs/gitbook/multiclass/iris.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/multiclass/iris.md b/docs/gitbook/multiclass/iris.md
new file mode 100644
index 0000000..e69de29

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/370e2aa3/docs/gitbook/multiclass/iris_dataset.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/multiclass/iris_dataset.md b/docs/gitbook/multiclass/iris_dataset.md
new file mode 100644
index 0000000..86f89ad
--- /dev/null
+++ b/docs/gitbook/multiclass/iris_dataset.md
@@ -0,0 +1,203 @@
+# Dataset prepration
+Iris Dataset: https://archive.ics.uci.edu/ml/datasets/Iris
+
+```sh
+$ wget http://archive.ics.uci.edu/ml/machine-learning-databases/iris/iris.data
+$ less iris.data
+
+   ...
+5.3,3.7,1.5,0.2,Iris-setosa
+5.0,3.3,1.4,0.2,Iris-setosa
+7.0,3.2,4.7,1.4,Iris-versicolor
+   ...
+```
+
+# Create training/test table in Hive
+
+```sql
+create database iris;
+use iris;
+
+create external table iris_raw (
+  rowid int,
+  label string,
+  features array<float>
+) ROW FORMAT DELIMITED FIELDS TERMINATED BY '|' COLLECTION ITEMS TERMINATED BY "," STORED AS TEXTFILE LOCATION '/dataset/iris/raw';
+```
+
+# Loading data into HDFS
+
+```sh
+$ awk -F"," 'NF >0 {OFS="|"; print NR,$5,$1","$2","$3","$4}' iris.data | head -3
+
+1|Iris-setosa|5.1,3.5,1.4,0.2
+2|Iris-setosa|4.9,3.0,1.4,0.2
+3|Iris-setosa|4.7,3.2,1.3,0.2
+```
+
+```sh
+$ awk -F"," 'NF >0 {OFS="|"; print NR,$5,$1","$2","$3","$4}' iris.data | hadoop fs -put - /dataset/iris/raw/iris.data
+```
+
+```sql
+select count(1) from iris_raw;
+
+> 150
+```
+
+# Feature scaling
+
+Normalization of feature weights is very important to get a good prediction in machine learning.
+
+```sql
+select 
+  min(features[0]), max(features[0]),
+  min(features[1]), max(features[1]),
+  min(features[2]), max(features[2]),
+  min(features[3]), max(features[3])
+from
+  iris_raw;
+
+> 4.3     7.9     2.0     4.4     1.0     6.9     0.1     2.5
+```
+
+```sql
+set hivevar:f0_min=4.3;
+set hivevar:f0_max=7.9;
+set hivevar:f1_min=2.0;
+set hivevar:f1_max=4.4;
+set hivevar:f2_min=1.0;
+set hivevar:f2_max=6.9;
+set hivevar:f3_min=0.1;
+set hivevar:f3_max=2.5;
+
+create or replace view iris_scaled
+as
+select
+  rowid, 
+  label,
+  add_bias(array(
+     concat("1:", rescale(features[0],${f0_min},${f0_max})), 
+     concat("2:", rescale(features[1],${f1_min},${f1_max})), 
+     concat("3:", rescale(features[2],${f2_min},${f2_max})), 
+     concat("4:", rescale(features[3],${f3_min},${f3_max}))
+  )) as features
+from 
+  iris_raw;
+```
+
+```sql
+select * from iris_scaled limit 3;
+
+> 1       Iris-setosa     ["1:0.22222215","2:0.625","3:0.0677966","4:0.041666664","0:1.0"]
+> 2       Iris-setosa     ["1:0.16666664","2:0.41666666","3:0.0677966","4:0.041666664","0:1.0"]
+> 3       Iris-setosa     ["1:0.11111101","2:0.5","3:0.05084745","4:0.041666664","0:1.0"]
+```
+
+_[LibSVM web page](http://www.csie.ntu.edu.tw/~cjlin/libsvmtools/datasets/multiclass.html#iris) provides a normalized (using [ZScore](https://github.com/myui/hivemall/wiki/Feature-scaling)) version of Iris dataset._
+
+# Create training/test data
+
+```sql
+set hivevar:rand_seed=31;
+
+create table iris_shuffled 
+as
+select rand(${rand_seed}) as rnd, * from iris_scaled;
+
+-- 80% for training
+create table train80p as
+select * from  iris_shuffled 
+order by rnd DESC
+limit 120;
+
+-- 20% for testing
+create table test20p as
+select * from  iris_shuffled 
+order by rnd ASC
+limit 30;
+
+create table test20p_exploded 
+as
+select 
+  rowid,
+  label,
+  extract_feature(feature) as feature,
+  extract_weight(feature) as value
+from 
+  test20p LATERAL VIEW explode(features) t AS feature;
+```
+
+# Define an amplified view for the training input
+```sql
+set hivevar:xtimes=10;
+set hivevar:shufflebuffersize=1000;
+
+create or replace view training_x10
+as
+select
+   rand_amplify(${xtimes}, ${shufflebuffersize}, rowid, label, features) as (rowid, label, features)
+from  
+   train80p;
+```
+
+# Training (multiclass classification)
+
+```sql
+create table model_scw1 as
+select 
+ label, 
+ feature,
+ argmin_kld(weight, covar) as weight
+from 
+ (select 
+     train_multiclass_scw(features, label) as (label, feature, weight, covar)
+  from 
+     training_x10
+ ) t 
+group by label, feature;
+```
+
+# Predict
+
+```sql
+create or replace view predict_scw1
+as
+select 
+  rowid, 
+  m.col0 as score, 
+  m.col1 as label
+from (
+select
+   rowid, 
+   maxrow(score, label) as m
+from (
+  select
+    t.rowid,
+    m.label,
+    sum(m.weight * t.value) as score
+  from 
+    test20p_exploded t LEFT OUTER JOIN
+    model_scw1 m ON (t.feature = m.feature)
+  group by
+    t.rowid, m.label
+) t1
+group by rowid
+) t2;
+```
+
+# Evaluation
+
+```sql
+create or replace view eval_scw1 as
+select 
+  t.label as actual, 
+  p.label as predicted
+from 
+  test20p t JOIN predict_scw1 p 
+    on (t.rowid = p.rowid);
+
+select count(1)/30 from eval_scw1 
+where actual = predicted;
+```
+> 0.9666666666666667

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/370e2aa3/docs/gitbook/multiclass/iris_randomforest.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/multiclass/iris_randomforest.md b/docs/gitbook/multiclass/iris_randomforest.md
new file mode 100644
index 0000000..bafa338
--- /dev/null
+++ b/docs/gitbook/multiclass/iris_randomforest.md
@@ -0,0 +1,307 @@
+*NOTE: RandomForest is being supported from Hivemall v0.4 or later.*
+
+# Dataset
+
+* https://archive.ics.uci.edu/ml/datasets/Iris
+
+```
+Attribute Information:
+   1. sepal length in cm
+   2. sepal width in cm
+   3. petal length in cm
+   4. petal width in cm
+   5. class: 
+      -- Iris Setosa
+      -- Iris Versicolour
+      -- Iris Virginica
+```
+
+# Table preparation
+
+```sql
+create database iris;
+use iris;
+
+create external table raw (
+  sepal_length int,
+  sepal_width int,
+  petal_length int,
+  petak_width int,
+  class string
+)
+ROW FORMAT DELIMITED
+  FIELDS TERMINATED BY ','
+  LINES TERMINATED BY '\n'
+STORED AS TEXTFILE LOCATION '/dataset/iris/raw';
+
+$ sed '/^$/d' iris.data | hadoop fs -put - /dataset/iris/raw/iris.data
+```
+
+```sql
+create table label_mapping 
+as
+select
+  class,
+  rank - 1 as label
+from (
+select
+  distinct class,
+  dense_rank() over (order by class) as rank
+from 
+  raw
+) t
+;
+```
+
+```sql
+create table training
+as
+select
+  rowid() as rowid,
+  array(t1.sepal_length, t1.sepal_width, t1.petal_length, t1.petak_width) as features,
+  t2.label
+from
+  raw t1
+  JOIN label_mapping t2 ON (t1.class = t2.class)
+;
+```
+
+# Training
+
+`train_randomforest_classifier` takes a dense `features` in double[] and a `label` starting from 0.
+
+```sql
+CREATE TABLE model 
+STORED AS SEQUENCEFILE 
+AS
+select 
+  train_randomforest_classifier(features, label) 
+  -- hivemall v0.4.1-alpha.2 and before
+  -- train_randomforest_classifier(features, label) as (pred_model, var_importance, oob_errors, oob_tests)
+  -- hivemall v0.4.1 and later
+  -- train_randomforest_classifier(features, label) as (model_id, model_type, pred_model, var_importance, oob_errors, oob_tests)
+from
+  training;
+```
+*Note: The default TEXTFILE should not be used for model table when using Javascript output through "-output javascript" option.*
+
+```
+hive> desc model;
+model_id                int                                         
+model_type              int                                         
+pred_model              string                                      
+var_importance          array<double>                               
+oob_errors              int                                         
+oob_tests               int  
+```
+
+## Training options
+
+"-help" option shows usage of the function.
+
+```
+select train_randomforest_classifier(features, label, "-help") from training;
+
+> FAILED: UDFArgumentException 
+usage: train_randomforest_classifier(double[] features, int label [,
+       string options]) - Returns a relation consists of <int model_id,
+       int model_type, string pred_model, array<double> var_importance,
+       int oob_errors, int oob_tests> [-attrs <arg>] [-depth <arg>]
+       [-disable_compression] [-help] [-leafs <arg>] [-output <arg>]
+       [-rule <arg>] [-seed <arg>] [-splits <arg>] [-trees <arg>] [-vars
+       <arg>]
+ -attrs,--attribute_types <arg>   Comma separated attribute types (Q for
+                                  quantitative variable and C for
+                                  categorical variable. e.g., [Q,C,Q,C])
+ -depth,--max_depth <arg>         The maximum number of the tree depth
+                                  [default: Integer.MAX_VALUE]
+ -disable_compression             Whether to disable compression of the
+                                  output script [default: false]
+ -help                            Show function help
+ -leafs,--max_leaf_nodes <arg>    The maximum number of leaf nodes
+                                  [default: Integer.MAX_VALUE]
+ -output,--output_type <arg>      The output type (serialization/ser or
+                                  opscode/vm or javascript/js) [default:
+                                  serialization]
+ -rule,--split_rule <arg>         Split algorithm [default: GINI, ENTROPY]
+ -seed <arg>                      seed value in long [default: -1
+                                  (random)]
+ -splits,--min_split <arg>        A node that has greater than or equals
+                                  to `min_split` examples will split
+                                  [default: 2]
+ -trees,--num_trees <arg>         The number of trees for each task
+                                  [default: 50]
+ -vars,--num_variables <arg>      The number of random selected features
+                                  [default: ceil(sqrt(x[0].length))].
+                                  int(num_variables * x[0].length) is
+                                  considered if num_variable is (0,1]
+```
+*Caution: "-num_trees" controls the number of trees for each task, not the total number of trees.*
+
+### Parallelize Training
+
+To parallelize RandomForest training, you can use UNION ALL as follows:
+
+```sql
+CREATE TABLE model 
+STORED AS SEQUENCEFILE 
+AS
+select 
+  train_randomforest_classifier(features, label, '-trees 25') 
+from
+  training
+UNION ALL
+select 
+  train_randomforest_classifier(features, label, '-trees 25')
+from
+  training
+;
+```
+
+### Learning stats
+
+[`Variable importance`](https://www.stat.berkeley.edu/~breiman/RandomForests/cc_home.htm#varimp) and [`Out Of Bag (OOB) error rate`](https://www.stat.berkeley.edu/~breiman/RandomForests/cc_home.htm#ooberr) of RandomForest can be shown as follows:
+
+```sql
+select
+  array_sum(var_importance) as var_importance,
+  sum(oob_errors) / sum(oob_tests) as oob_err_rate
+from
+  model;
+```
+> [2.81010338879605,0.4970357753626371,23.790369091407698,14.315316390235273]     0.05333333333333334
+
+### Output prediction model by Javascipt
+
+```sql
+CREATE TABLE model_javascript
+STORED AS SEQUENCEFILE 
+AS
+select train_randomforest_classifier(features, label, "-output_type js -disable_compression")
+from training;
+
+select model from model_javascript limit 1;
+```
+
+```js
+if(x[3] <= 0.5) {
+  0;
+} else  {
+  if(x[2] <= 4.5) {
+    if(x[3] <= 1.5) {
+      if(x[0] <= 4.5) {
+        1;
+      } else  {
+        if(x[0] <= 5.5) {
+          1;
+        } else  {
+          if(x[1] <= 2.5) {
+            1;
+          } else  {
+            1;
+          }
+        }
+      }
+    } else  {
+      2;
+    }
+  } else  {
+    if(x[3] <= 1.5) {
+      2;
+    } else  {
+      2;
+    }
+  }
+}
+```
+
+# Prediction
+
+```sql
+set hivevar:classification=true;
+set hive.auto.convert.join=true;
+set hive.mapjoin.optimized.hashtable=false;
+
+create table predicted_vm
+as
+SELECT
+  rowid,
+  rf_ensemble(predicted) as predicted
+FROM (
+  SELECT
+    rowid, 
+    -- hivemall v0.4.1-alpha.2 and before
+    -- tree_predict(p.model, t.features, ${classification}) as predicted
+    -- hivemall v0.4.1 and later
+    tree_predict(p.model_id, p.model_type, p.pred_model, t.features, ${classification}) as predicted
+  FROM
+    model p
+    LEFT OUTER JOIN -- CROSS JOIN
+    training t
+) t1
+group by
+  rowid
+;
+```
+_Note: Javascript outputs can be evaluated by `js_tree_predict`._
+
+### Parallelize Prediction
+
+The following query runs predictions in N-parallel. It would reduce elapsed time for prediction almost by N.
+
+```sql
+SET hivevar:classification=true;
+set hive.auto.convert.join=true;
+SET hive.mapjoin.optimized.hashtable=false;
+SET mapred.reduce.tasks=8;
+
+create table predicted_vm
+as
+SELECT
+  rowid,
+  rf_ensemble(predicted) as predicted
+FROM (
+  SELECT
+    t.rowid, 
+    -- hivemall v0.4.1-alpha.2 and before
+    -- tree_predict(p.pred_model, t.features, ${classification}) as predicted
+    -- hivemall v0.4.1 and later
+    tree_predict(p.model_id, p.model_type, p.pred_model, t.features, ${classification}) as predicted
+  FROM (
+    SELECT model_id, model_type, pred_model
+    FROM model
+    DISTRIBUTE BY rand(1)
+  ) p 
+  LEFT OUTER JOIN training t
+) t1
+group by
+  rowid
+;
+```
+
+# Evaluation
+
+```sql
+select count(1) from training;
+> 150
+
+set hivevar:total_cnt=150;
+
+WITH t1 as (
+SELECT
+  t.rowid,
+  t.label as actual,
+  p.predicted.label as predicted
+FROM
+  predicted_vm p
+  LEFT OUTER JOIN training t ON (t.rowid = p.rowid)
+)
+SELECT
+  count(1) / ${total_cnt}
+FROM
+  t1
+WHERE
+  actual = predicted
+;
+```
+> 0.9533333333333334
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/370e2aa3/docs/gitbook/multiclass/iris_scw.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/multiclass/iris_scw.md b/docs/gitbook/multiclass/iris_scw.md
new file mode 100644
index 0000000..bafa338
--- /dev/null
+++ b/docs/gitbook/multiclass/iris_scw.md
@@ -0,0 +1,307 @@
+*NOTE: RandomForest is being supported from Hivemall v0.4 or later.*
+
+# Dataset
+
+* https://archive.ics.uci.edu/ml/datasets/Iris
+
+```
+Attribute Information:
+   1. sepal length in cm
+   2. sepal width in cm
+   3. petal length in cm
+   4. petal width in cm
+   5. class: 
+      -- Iris Setosa
+      -- Iris Versicolour
+      -- Iris Virginica
+```
+
+# Table preparation
+
+```sql
+create database iris;
+use iris;
+
+create external table raw (
+  sepal_length int,
+  sepal_width int,
+  petal_length int,
+  petak_width int,
+  class string
+)
+ROW FORMAT DELIMITED
+  FIELDS TERMINATED BY ','
+  LINES TERMINATED BY '\n'
+STORED AS TEXTFILE LOCATION '/dataset/iris/raw';
+
+$ sed '/^$/d' iris.data | hadoop fs -put - /dataset/iris/raw/iris.data
+```
+
+```sql
+create table label_mapping 
+as
+select
+  class,
+  rank - 1 as label
+from (
+select
+  distinct class,
+  dense_rank() over (order by class) as rank
+from 
+  raw
+) t
+;
+```
+
+```sql
+create table training
+as
+select
+  rowid() as rowid,
+  array(t1.sepal_length, t1.sepal_width, t1.petal_length, t1.petak_width) as features,
+  t2.label
+from
+  raw t1
+  JOIN label_mapping t2 ON (t1.class = t2.class)
+;
+```
+
+# Training
+
+`train_randomforest_classifier` takes a dense `features` in double[] and a `label` starting from 0.
+
+```sql
+CREATE TABLE model 
+STORED AS SEQUENCEFILE 
+AS
+select 
+  train_randomforest_classifier(features, label) 
+  -- hivemall v0.4.1-alpha.2 and before
+  -- train_randomforest_classifier(features, label) as (pred_model, var_importance, oob_errors, oob_tests)
+  -- hivemall v0.4.1 and later
+  -- train_randomforest_classifier(features, label) as (model_id, model_type, pred_model, var_importance, oob_errors, oob_tests)
+from
+  training;
+```
+*Note: The default TEXTFILE should not be used for model table when using Javascript output through "-output javascript" option.*
+
+```
+hive> desc model;
+model_id                int                                         
+model_type              int                                         
+pred_model              string                                      
+var_importance          array<double>                               
+oob_errors              int                                         
+oob_tests               int  
+```
+
+## Training options
+
+"-help" option shows usage of the function.
+
+```
+select train_randomforest_classifier(features, label, "-help") from training;
+
+> FAILED: UDFArgumentException 
+usage: train_randomforest_classifier(double[] features, int label [,
+       string options]) - Returns a relation consists of <int model_id,
+       int model_type, string pred_model, array<double> var_importance,
+       int oob_errors, int oob_tests> [-attrs <arg>] [-depth <arg>]
+       [-disable_compression] [-help] [-leafs <arg>] [-output <arg>]
+       [-rule <arg>] [-seed <arg>] [-splits <arg>] [-trees <arg>] [-vars
+       <arg>]
+ -attrs,--attribute_types <arg>   Comma separated attribute types (Q for
+                                  quantitative variable and C for
+                                  categorical variable. e.g., [Q,C,Q,C])
+ -depth,--max_depth <arg>         The maximum number of the tree depth
+                                  [default: Integer.MAX_VALUE]
+ -disable_compression             Whether to disable compression of the
+                                  output script [default: false]
+ -help                            Show function help
+ -leafs,--max_leaf_nodes <arg>    The maximum number of leaf nodes
+                                  [default: Integer.MAX_VALUE]
+ -output,--output_type <arg>      The output type (serialization/ser or
+                                  opscode/vm or javascript/js) [default:
+                                  serialization]
+ -rule,--split_rule <arg>         Split algorithm [default: GINI, ENTROPY]
+ -seed <arg>                      seed value in long [default: -1
+                                  (random)]
+ -splits,--min_split <arg>        A node that has greater than or equals
+                                  to `min_split` examples will split
+                                  [default: 2]
+ -trees,--num_trees <arg>         The number of trees for each task
+                                  [default: 50]
+ -vars,--num_variables <arg>      The number of random selected features
+                                  [default: ceil(sqrt(x[0].length))].
+                                  int(num_variables * x[0].length) is
+                                  considered if num_variable is (0,1]
+```
+*Caution: "-num_trees" controls the number of trees for each task, not the total number of trees.*
+
+### Parallelize Training
+
+To parallelize RandomForest training, you can use UNION ALL as follows:
+
+```sql
+CREATE TABLE model 
+STORED AS SEQUENCEFILE 
+AS
+select 
+  train_randomforest_classifier(features, label, '-trees 25') 
+from
+  training
+UNION ALL
+select 
+  train_randomforest_classifier(features, label, '-trees 25')
+from
+  training
+;
+```
+
+### Learning stats
+
+[`Variable importance`](https://www.stat.berkeley.edu/~breiman/RandomForests/cc_home.htm#varimp) and [`Out Of Bag (OOB) error rate`](https://www.stat.berkeley.edu/~breiman/RandomForests/cc_home.htm#ooberr) of RandomForest can be shown as follows:
+
+```sql
+select
+  array_sum(var_importance) as var_importance,
+  sum(oob_errors) / sum(oob_tests) as oob_err_rate
+from
+  model;
+```
+> [2.81010338879605,0.4970357753626371,23.790369091407698,14.315316390235273]     0.05333333333333334
+
+### Output prediction model by Javascipt
+
+```sql
+CREATE TABLE model_javascript
+STORED AS SEQUENCEFILE 
+AS
+select train_randomforest_classifier(features, label, "-output_type js -disable_compression")
+from training;
+
+select model from model_javascript limit 1;
+```
+
+```js
+if(x[3] <= 0.5) {
+  0;
+} else  {
+  if(x[2] <= 4.5) {
+    if(x[3] <= 1.5) {
+      if(x[0] <= 4.5) {
+        1;
+      } else  {
+        if(x[0] <= 5.5) {
+          1;
+        } else  {
+          if(x[1] <= 2.5) {
+            1;
+          } else  {
+            1;
+          }
+        }
+      }
+    } else  {
+      2;
+    }
+  } else  {
+    if(x[3] <= 1.5) {
+      2;
+    } else  {
+      2;
+    }
+  }
+}
+```
+
+# Prediction
+
+```sql
+set hivevar:classification=true;
+set hive.auto.convert.join=true;
+set hive.mapjoin.optimized.hashtable=false;
+
+create table predicted_vm
+as
+SELECT
+  rowid,
+  rf_ensemble(predicted) as predicted
+FROM (
+  SELECT
+    rowid, 
+    -- hivemall v0.4.1-alpha.2 and before
+    -- tree_predict(p.model, t.features, ${classification}) as predicted
+    -- hivemall v0.4.1 and later
+    tree_predict(p.model_id, p.model_type, p.pred_model, t.features, ${classification}) as predicted
+  FROM
+    model p
+    LEFT OUTER JOIN -- CROSS JOIN
+    training t
+) t1
+group by
+  rowid
+;
+```
+_Note: Javascript outputs can be evaluated by `js_tree_predict`._
+
+### Parallelize Prediction
+
+The following query runs predictions in N-parallel. It would reduce elapsed time for prediction almost by N.
+
+```sql
+SET hivevar:classification=true;
+set hive.auto.convert.join=true;
+SET hive.mapjoin.optimized.hashtable=false;
+SET mapred.reduce.tasks=8;
+
+create table predicted_vm
+as
+SELECT
+  rowid,
+  rf_ensemble(predicted) as predicted
+FROM (
+  SELECT
+    t.rowid, 
+    -- hivemall v0.4.1-alpha.2 and before
+    -- tree_predict(p.pred_model, t.features, ${classification}) as predicted
+    -- hivemall v0.4.1 and later
+    tree_predict(p.model_id, p.model_type, p.pred_model, t.features, ${classification}) as predicted
+  FROM (
+    SELECT model_id, model_type, pred_model
+    FROM model
+    DISTRIBUTE BY rand(1)
+  ) p 
+  LEFT OUTER JOIN training t
+) t1
+group by
+  rowid
+;
+```
+
+# Evaluation
+
+```sql
+select count(1) from training;
+> 150
+
+set hivevar:total_cnt=150;
+
+WITH t1 as (
+SELECT
+  t.rowid,
+  t.label as actual,
+  p.predicted.label as predicted
+FROM
+  predicted_vm p
+  LEFT OUTER JOIN training t ON (t.rowid = p.rowid)
+)
+SELECT
+  count(1) / ${total_cnt}
+FROM
+  t1
+WHERE
+  actual = predicted
+;
+```
+> 0.9533333333333334
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/370e2aa3/docs/gitbook/multiclass/news20.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/multiclass/news20.md b/docs/gitbook/multiclass/news20.md
new file mode 100644
index 0000000..e69de29

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/370e2aa3/docs/gitbook/multiclass/news20_dataset.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/multiclass/news20_dataset.md b/docs/gitbook/multiclass/news20_dataset.md
new file mode 100644
index 0000000..35ada12
--- /dev/null
+++ b/docs/gitbook/multiclass/news20_dataset.md
@@ -0,0 +1,77 @@
+Get the news20 dataset.
+http://www.csie.ntu.edu.tw/~cjlin/libsvmtools/datasets/multiclass.html#news20
+
+```sh
+$ cat <<EOF > conv.awk
+BEGIN{ FS=" " }
+{
+    label=\$1;
+    features=\$2;
+    for(i=3;i<=NF;i++)
+    {
+        features = features "," \$i;
+    }
+    print NR "\t" label "\t" features;
+}
+END{}
+EOF
+
+$ gawk -f conv.awk news20.scale > news20_scale.train
+$ gawk -f conv.awk news20.t.scale > news20_scale.test
+```
+
+## Putting data on HDFS
+```sh
+hadoop fs -mkdir -p /dataset/news20-multiclass/train
+hadoop fs -mkdir -p /dataset/news20-multiclass/test
+
+hadoop fs -copyFromLocal news20_scale.train /dataset/news20-multiclass/train
+hadoop fs -copyFromLocal news20_scale.test /dataset/news20-multiclass/test
+```
+
+## Training/test data prepareation
+```sql
+use news20;
+
+delete jar /home/myui/tmp/hivemall.jar;
+add jar /home/myui/tmp/hivemall.jar;
+
+source /home/myui/tmp/define-all.hive;
+
+Create external table news20mc_train (
+  rowid int,
+  label int,
+  features ARRAY<STRING>
+) ROW FORMAT DELIMITED FIELDS TERMINATED BY '\t' COLLECTION ITEMS TERMINATED BY "," STORED AS TEXTFILE LOCATION '/dataset/news20-multiclass/train';
+
+Create external table news20mc_test (
+  rowid int, 
+  label int,
+  features ARRAY<STRING>
+) ROW FORMAT DELIMITED FIELDS TERMINATED BY '\t' COLLECTION ITEMS TERMINATED BY "," STORED AS TEXTFILE LOCATION '/dataset/news20-multiclass/test';
+
+set hivevar:seed=31;
+create or replace view news20mc_train_x3
+as
+select 
+  * 
+from (
+select
+   amplify(3, *) as (rowid, label, features)
+from  
+   news20mc_train 
+) t
+CLUSTER BY rand(${seed});
+
+create table news20mc_test_exploded as
+select 
+  rowid,
+  label,
+  cast(split(feature,":")[0] as int) as feature,
+  cast(split(feature,":")[1] as float) as value
+  -- hivemall v0.3.1 or later
+  -- cast(extract_feature(feature) as int) as feature,
+  -- extract_weight(feature) as value
+from 
+  news20mc_test LATERAL VIEW explode(addBias(features)) t AS feature;
+```
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/370e2aa3/docs/gitbook/multiclass/news20_ensemble.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/multiclass/news20_ensemble.md b/docs/gitbook/multiclass/news20_ensemble.md
new file mode 100644
index 0000000..9cfd35d
--- /dev/null
+++ b/docs/gitbook/multiclass/news20_ensemble.md
@@ -0,0 +1,180 @@
+This example explains how to run ensemble learning in Hivemall.   
+Two heads are better than one? Let's verify it by ensemble learning.
+
+---
+
+## UDF preparation
+```sql
+delete jar /home/myui/tmp/hivemall.jar;
+add jar /home/myui/tmp/hivemall.jar;
+
+source /home/myui/tmp/define-all.hive;
+```
+
+[Case1] Model ensemble/mixing
+=======================
+
+## training
+```sql
+SET hive.exec.parallel=true;
+SET hive.exec.parallel.thread.number=8;
+SET mapred.reduce.tasks=4;
+
+drop table news20mc_ensemble_model1;
+create table news20mc_ensemble_model1 as
+select 
+ label, 
+ -- cast(feature as int) as feature, -- hivemall v0.1
+ argmin_kld(feature, covar) as feature, -- hivemall v0.2 or later
+ voted_avg(weight) as weight
+from 
+ (select 
+     -- train_multiclass_cw(addBias(features),label) as (label,feature,weight)      -- hivemall v0.1
+     train_multiclass_cw(addBias(features),label) as (label,feature,weight,covar)   -- hivemall v0.2 or later
+  from 
+     news20mc_train_x3
+  union all
+  select 
+     -- train_multiclass_arow(addBias(features),label) as (label,feature,weight)    -- hivemall v0.1
+     train_multiclass_arow(addBias(features),label) as (label,feature,weight,covar) -- hivemall v0.2 or later
+  from 
+     news20mc_train_x3
+  union all
+  select 
+     -- train_multiclass_scw(addBias(features),label) as (label,feature,weight)     -- hivemall v0.1
+     train_multiclass_scw(addBias(features),label) as (label,feature,weight,covar)  -- hivemall v0.2 or later
+  from 
+     news20mc_train_x3
+ ) t 
+group by label, feature;
+
+-- reset to the default
+SET hive.exec.parallel=false;
+SET mapred.reduce.tasks=-1;
+```
+
+## prediction
+```sql
+create or replace view news20mc_ensemble_predict1 
+as
+select 
+  rowid, 
+  m.col0 as score, 
+  m.col1 as label
+from (
+select
+   rowid, 
+   maxrow(score, label) as m
+from (
+  select
+    t.rowid,
+    m.label,
+    sum(m.weight * t.value) as score
+  from 
+    news20mc_test_exploded t LEFT OUTER JOIN
+    news20mc_ensemble_model1 m ON (t.feature = m.feature)
+  group by
+    t.rowid, m.label
+) t1
+group by rowid
+) t2;
+```
+
+## evaluation
+```sql
+create or replace view news20mc_ensemble_submit1 as
+select 
+  t.label as actual, 
+  pd.label as predicted
+from 
+  news20mc_test t JOIN news20mc_ensemble_predict1 pd 
+    on (t.rowid = pd.rowid);
+```
+
+```
+select count(1)/3993 from news20mc_ensemble_submit1 
+where actual == predicted;
+```
+
+> 0.8494866015527173
+
+## Cleaning
+
+```sql
+drop table news20mc_ensemble_model1;
+drop view news20mc_ensemble_predict1;
+drop view news20mc_ensemble_submit1;
+```
+---
+
+Unfortunately, too many cooks spoil the broth in this case :-(
+
+| Algorithm | Accuracy |
+|:-----------|------------:|
+| AROW | 0.8474830954169797 |
+| SCW2 |  0.8482344102178813 |
+| Ensemble(model) | 0.8494866015527173 |
+| CW |  0.850488354620586 |
+
+
+---
+
+[Case2] Prediction ensemble
+=================
+
+## prediction
+```sql
+create or replace view news20mc_pred_ensemble_predict1 
+as
+select 
+  rowid, 
+  m.col1 as label
+from (
+  select
+    rowid, 
+    maxrow(cnt, label) as m
+  from (
+    select
+      rowid,
+      label,
+      count(1) as cnt
+    from (
+      select * from news20mc_arow_predict1
+      union all
+      select * from news20mc_scw2_predict1
+      union all
+      select * from news20mc_cw_predict1
+    ) t1
+    group by rowid, label
+  ) t2
+  group by rowid
+) t3;
+```
+
+## evaluation
+```sql
+create or replace view news20mc_pred_ensemble_submit1 as
+select 
+  t.label as actual, 
+  pd.label as predicted
+from 
+  news20mc_test t JOIN news20mc_pred_ensemble_predict1 pd 
+    on (t.rowid = pd.rowid);
+```
+
+```
+select count(1)/3993 from news20mc_pred_ensemble_submit1 
+where actual == predicted;
+```
+
+> 0.8499874780866516
+
+Unfortunately, too many cooks spoil the broth in this case too :-(
+
+| Algorithm | Accuracy |
+|:-----------|------------:|
+| AROW | 0.8474830954169797 |
+| SCW2 |  0.8482344102178813 |
+| Ensemble(model) | 0.8494866015527173 |
+| Ensemble(prediction) | 0.8499874780866516 |
+| CW |  0.850488354620586 |
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/370e2aa3/docs/gitbook/multiclass/news20_one-vs-the-rest.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/multiclass/news20_one-vs-the-rest.md b/docs/gitbook/multiclass/news20_one-vs-the-rest.md
new file mode 100644
index 0000000..4c611d0
--- /dev/null
+++ b/docs/gitbook/multiclass/news20_one-vs-the-rest.md
@@ -0,0 +1,330 @@
+A one-vs-the-rest classifier use the binary classifier for each class.
+
+## UDF preparation
+```sql
+delete jar /home/myui/tmp/hivemall.jar;
+add jar /home/myui/tmp/hivemall.jar;
+
+source /home/myui/tmp/define-all.hive;
+```
+
+## training
+```sql
+SET mapred.reduce.tasks=4;
+
+drop table news20_onevsrest_arow_model;
+create table news20_onevsrest_arow_model 
+as
+select
+  label,
+  feature,
+  -- voted_avg(weight) as weight -- [hivemall v0.1]
+  argmin_kld(weight, covar) as weight -- [hivemall v0.2 or later]
+from (
+select
+  1 as label,
+  *
+from (
+select 
+  -- train_arow(features, target) as (feature, weight)     -- [hivemall v0.1]
+  train_arow(features, target) as (feature, weight, covar) -- [hivemall v0.2 or later]
+from 
+  news20_onevsrest_train_x3
+where
+  label = 1
+) t1
+union all
+select
+  2 as label,
+  *
+from (
+select 
+  train_arow(features, target) as (feature, weight, covar)
+from 
+  news20_onevsrest_train_x3
+where
+  label = 2
+) t2
+union all
+select
+  3 as label,
+  *
+from (
+select 
+  train_arow(features, target) as (feature, weight, covar)
+from 
+  news20_onevsrest_train_x3
+where
+  label = 3
+) t3
+union all
+select
+  4 as label,
+  *
+from (
+select 
+  train_arow(features, target) as (feature, weight, covar)
+from 
+  news20_onevsrest_train_x3
+where
+  label = 4
+) t4
+union all
+select
+  5 as label,
+  *
+from (
+select 
+  train_arow(features, target) as (feature, weight, covar)
+from 
+  news20_onevsrest_train_x3
+where
+  label = 5
+) t5
+union all
+select
+  6 as label,
+  *
+from (
+select 
+  train_arow(features, target) as (feature, weight, covar)
+from 
+  news20_onevsrest_train_x3
+where
+  label = 6
+) t6
+union all
+select
+  7 as label,
+  *
+from (
+select 
+  train_arow(features, target) as (feature, weight, covar)
+from 
+  news20_onevsrest_train_x3
+where
+  label = 7
+) t7
+union all
+select
+  8 as label,
+  *
+from (
+select 
+  train_arow(features, target) as (feature, weight, covar)
+from 
+  news20_onevsrest_train_x3
+where
+  label = 8
+) t8
+union all
+select
+  9 as label,
+  *
+from (
+select 
+  train_arow(features, target) as (feature, weight, covar)
+from 
+  news20_onevsrest_train_x3
+where
+  label = 9
+) t9
+union all
+select
+  10 as label,
+  *
+from (
+select 
+  train_arow(features, target) as (feature, weight, covar)
+from 
+  news20_onevsrest_train_x3
+where
+  label = 10
+) t10
+union all
+select
+  11 as label,
+  *
+from (
+select 
+  train_arow(features, target) as (feature, weight, covar)
+from 
+  news20_onevsrest_train_x3
+where
+  label = 11
+) t11
+union all
+select
+  12 as label,
+  *
+from (
+select 
+  train_arow(features, target) as (feature, weight, covar)
+from 
+  news20_onevsrest_train_x3
+where
+  label = 12
+) t12
+union all
+select
+  13 as label,
+  *
+from (
+select 
+  train_arow(features, target) as (feature, weight, covar)
+from 
+  news20_onevsrest_train_x3
+where
+  label = 13
+) t13
+union all
+select
+  14 as label,
+  *
+from (
+select 
+  train_arow(features, target) as (feature, weight, covar)
+from 
+  news20_onevsrest_train_x3
+where
+  label = 14
+) t14
+union all
+select
+  15 as label,
+  *
+from (
+select 
+  train_arow(features, target) as (feature, weight, covar)
+from 
+  news20_onevsrest_train_x3
+where
+  label = 15
+) t15
+union all
+select
+  16 as label,
+  *
+from (
+select 
+  train_arow(features, target) as (feature, weight, covar)
+from 
+  news20_onevsrest_train_x3
+where
+  label = 16
+) t16
+union all
+select
+  17 as label,
+  *
+from (
+select 
+  train_arow(features, target) as (feature, weight, covar)
+from 
+  news20_onevsrest_train_x3
+where
+  label = 17
+) t17
+union all
+select
+  18 as label,
+  *
+from (
+select 
+  train_arow(features, target) as (feature, weight, covar)
+from 
+  news20_onevsrest_train_x3
+where
+  label = 18
+) t18
+union all
+select
+  19 as label,
+  *
+from (
+select 
+  train_arow(features, target) as (feature, weight, covar)
+from 
+  news20_onevsrest_train_x3
+where
+  label = 19
+) t19
+union all
+select
+  20 as label,
+  *
+from (
+select 
+  train_arow(features, target) as (feature, weight, covar)
+from 
+  news20_onevsrest_train_x3
+where
+  label = 20
+) t20
+) t
+group by 
+  label, feature;
+
+-- reset to the default
+SET mapred.reduce.tasks=-1;
+```
+Note that the above query is optimized to scan news20_onevsrest_train_x3 once!
+
+## prediction
+```sql
+create or replace view news20_onevsrest_arow_predict 
+as
+select 
+  rowid, 
+  m.col0 as score, 
+  m.col1 as label
+from (
+select
+   rowid, 
+   maxrow(score, label) as m
+from (
+  select
+    t.rowid,
+    m.label,
+    sum(m.weight * t.value) as score
+  from 
+    news20mc_test_exploded t LEFT OUTER JOIN
+    news20_onevsrest_arow_model m ON (t.feature = m.feature)
+  group by
+    t.rowid, m.label
+) t1
+group by rowid
+) t2;
+```
+
+## evaluation
+```sql
+create or replace view news20_onevsrest_arow_submit as
+select 
+  t.label as actual, 
+  pd.label as predicted
+from 
+  news20mc_test t JOIN news20_onevsrest_arow_predict pd 
+    on (t.rowid = pd.rowid);
+```
+
+```
+select count(1)/3993 from news20_onevsrest_arow_submit
+where actual == predicted;
+```
+
+> 0.8567493112947658
+
+## Cleaning
+
+```sql
+drop table news20_onevsrest_arow_model1;
+drop view news20_onevsrest_arow_predict1;
+drop view news20_onevsrest_arow_submit1;
+```
+
+| Algorithm | Accuracy |
+|:-----------|------------:|
+| AROW(multi-class) | 0.8474830954169797 |
+| CW |  0.850488354620586 |
+| AROW(one-vs-rest) | 0.8567493112947658 |

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/370e2aa3/docs/gitbook/multiclass/news20_one-vs-the-rest_dataset.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/multiclass/news20_one-vs-the-rest_dataset.md b/docs/gitbook/multiclass/news20_one-vs-the-rest_dataset.md
new file mode 100644
index 0000000..2a69615
--- /dev/null
+++ b/docs/gitbook/multiclass/news20_one-vs-the-rest_dataset.md
@@ -0,0 +1,52 @@
+*One-vs-the-rest* is a multiclass classification method that uses binary classifiers independently for each class.
+http://en.wikipedia.org/wiki/Multiclass_classification#one_vs_all
+
+## UDF preparation
+```sql
+delete jar /home/myui/tmp/hivemall.jar;
+add jar /home/myui/tmp/hivemall.jar;
+
+source /home/myui/tmp/define-all.hive;
+```
+
+## Dataset preparation for one-vs-the-rest classifiers
+
+```sql
+select collect_set(label) from news20mc_train;
+```
+> [1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,17,16,19,18,20]
+
+```sql
+SET hivevar:possible_labels="1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,17,16,19,18,20";
+```
+
+[one-vs-rest.awk](https://github.com/myui/hivemall/blob/master/resources/misc/one-vs-rest.awk)
+
+```
+create or replace view news20_onevsrest_train
+as
+select transform(${possible_labels}, rowid, label, addBias(features))
+  ROW FORMAT DELIMITED
+    FIELDS TERMINATED BY "\t"
+    COLLECTION ITEMS TERMINATED BY ","
+    LINES TERMINATED BY "\n"
+using 'gawk -f one-vs-rest.awk'
+  as (rowid BIGINT, label INT, target INT, features ARRAY<STRING>)
+  ROW FORMAT DELIMITED
+    FIELDS TERMINATED BY "\t"
+    COLLECTION ITEMS TERMINATED BY ","
+    LINES TERMINATED BY "\n"
+from news20mc_train;
+
+create or replace view news20_onevsrest_train_x3
+as
+select
+ *
+from (
+  select
+    amplify(3, *) as (rowid, label, target, features)
+  from
+    news20_onevsrest_train
+) t
+CLUSTER BY rand();
+```

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/370e2aa3/docs/gitbook/multiclass/news20_pa.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/multiclass/news20_pa.md b/docs/gitbook/multiclass/news20_pa.md
new file mode 100644
index 0000000..8e69beb
--- /dev/null
+++ b/docs/gitbook/multiclass/news20_pa.md
@@ -0,0 +1,90 @@
+Preparation
+=========
+
+## UDF preparation
+```
+delete jar /home/myui/tmp/hivemall.jar;
+add jar /home/myui/tmp/hivemall.jar;
+
+source /home/myui/tmp/define-all.hive;
+```
+
+---
+#[Passive Aggressive (PA2)]
+
+Training
+======
+
+## model building
+```sql
+drop table news20mc_pa2_model1;
+create table news20mc_pa2_model1 as
+select 
+ label, 
+ cast(feature as int) as feature,
+ voted_avg(weight) as weight
+from 
+ (select 
+     train_multiclass_pa2(addBias(features),label) as (label,feature,weight)
+  from 
+     news20mc_train_x3
+ ) t 
+group by label, feature;
+```
+
+## prediction
+```
+create or replace view news20mc_pa2_predict1 
+as
+select 
+  rowid, 
+  m.col0 as score, 
+  m.col1 as label
+from (
+select
+   rowid, 
+   maxrow(score, label) as m
+from (
+  select
+    t.rowid,
+    m.label,
+    sum(m.weight * t.value) as score
+  from 
+    news20mc_test_exploded t LEFT OUTER JOIN
+    news20mc_pa2_model1 m ON (t.feature = m.feature)
+  group by
+    t.rowid, m.label
+) t1
+group by rowid
+) t2;
+```
+
+## evaluation
+```sql
+create or replace view news20mc_pa2_submit1 as
+select 
+  t.label as actual, 
+  pd.label as predicted
+from 
+  news20mc_test t JOIN news20mc_pa2_predict1 pd 
+    on (t.rowid = pd.rowid);
+```
+
+```sql
+select count(1)/3993 from news20mc_pa2_submit1 
+where actual == predicted;
+```
+
+> 0.7478086651640371 (plain)
+
+> 0.8204357625845229 (x3)
+
+> 0.8204357625845229 (x3 + bagging)
+
+## Cleaning
+
+```sql
+drop table news20mc_pa2_model1;
+drop table news20mc_pa2_predict1;
+drop view news20mc_pa2_submit1;
+```
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/370e2aa3/docs/gitbook/multiclass/news20_scw.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/multiclass/news20_scw.md b/docs/gitbook/multiclass/news20_scw.md
new file mode 100644
index 0000000..330c163
--- /dev/null
+++ b/docs/gitbook/multiclass/news20_scw.md
@@ -0,0 +1,319 @@
+| Algorithm | Accuracy |
+|:-----------|------------:|
+| PA2 | 0.8204357625845229 |
+| SCW1 | 0.8314550463310794 |
+| AROW | 0.8474830954169797 |
+| SCW2 |  0.8482344102178813 |
+| CW |  0.850488354620586 |
+---
+
+Preparation
+=========
+
+## UDF preparation
+```sql
+delete jar /home/myui/tmp/hivemall.jar;
+add jar /home/myui/tmp/hivemall.jar;
+
+source /home/myui/tmp/define-all.hive;
+```
+
+---
+#[CW]
+
+## training
+```sql
+drop table news20mc_cw_model1;
+create table news20mc_cw_model1 as
+select 
+ label, 
+ cast(feature as int) as feature,
+ -- voted_avg(weight) as weight -- [hivemall v0.1]
+ argmin_kld(weight, covar) as weight -- [hivemall v0.2 or later]
+from 
+ (select 
+     -- train_multiclass_cw(addBias(features),label) as (label,feature,weight) -- [hivemall v0.1]
+     train_multiclass_cw(addBias(features),label) as (label,feature,weight,covar)    -- [hivemall v0.2 or later]
+  from 
+     news20mc_train_x3
+ ) t 
+group by label, feature;
+```
+
+## prediction
+```sql
+create or replace view news20mc_cw_predict1 
+as
+select 
+  rowid, 
+  m.col0 as score, 
+  m.col1 as label
+from (
+select
+   rowid, 
+   maxrow(score, label) as m
+from (
+  select
+    t.rowid,
+    m.label,
+    sum(m.weight * t.value) as score
+  from 
+    news20mc_test_exploded t LEFT OUTER JOIN
+    news20mc_cw_model1 m ON (t.feature = m.feature)
+  group by
+    t.rowid, m.label
+) t1
+group by rowid
+) t2;
+```
+
+## evaluation
+```sql
+create or replace view news20mc_cw_submit1 as
+select 
+  t.label as actual, 
+  pd.label as predicted
+from 
+  news20mc_test t JOIN news20mc_cw_predict1 pd 
+    on (t.rowid = pd.rowid);
+```
+
+```
+select count(1)/3993 from news20mc_cw_submit1 
+where actual == predicted;
+```
+
+> 0.850488354620586
+
+## Cleaning
+
+```sql
+drop table news20mc_cw_model1;
+drop table news20mc_cw_predict1;
+drop view news20mc_cw_submit1;
+```
+
+---
+#[AROW]
+
+## training
+```sql
+drop table news20mc_arow_model1;
+create table news20mc_arow_model1 as
+select 
+ label, 
+ cast(feature as int) as feature,
+ -- voted_avg(weight) as weight -- [hivemall v0.1]
+ argmin_kld(weight, covar) as weight -- [hivemall v0.2 or later]
+from 
+ (select 
+     -- train_multiclass_arow(addBias(features),label) as (label,feature,weight) -- [hivemall v0.1]
+     train_multiclass_arow(addBias(features),label) as (label,feature,weight,covar) -- [hivemall v0.2 or later]
+  from 
+     news20mc_train_x3
+ ) t 
+group by label, feature;
+```
+
+## prediction
+```sql
+create or replace view news20mc_arow_predict1 
+as
+select 
+  rowid, 
+  m.col0 as score, 
+  m.col1 as label
+from (
+select
+   rowid, 
+   maxrow(score, label) as m
+from (
+  select
+    t.rowid,
+    m.label,
+    sum(m.weight * t.value) as score
+  from 
+    news20mc_test_exploded t LEFT OUTER JOIN
+    news20mc_arow_model1 m ON (t.feature = m.feature)
+  group by
+    t.rowid, m.label
+) t1
+group by rowid
+) t2;
+```
+
+## evaluation
+```sql
+create or replace view news20mc_arow_submit1 as
+select 
+  t.label as actual, 
+  pd.label as predicted
+from 
+  news20mc_test t JOIN news20mc_arow_predict1 pd 
+    on (t.rowid = pd.rowid);
+```
+
+```
+select count(1)/3993 from news20mc_arow_submit1 
+where actual == predicted;
+```
+
+> 0.8474830954169797
+
+## Cleaning
+
+```sql
+drop table news20mc_arow_model1;
+drop table news20mc_arow_predict1;
+drop view news20mc_arow_submit1;
+```
+
+---
+#[SCW1]
+
+## training
+```sql
+drop table news20mc_scw_model1;
+create table news20mc_scw_model1 as
+select 
+ label, 
+ cast(feature as int) as feature,
+ -- voted_avg(weight) as weight -- [hivemall v0.1]
+ argmin_kld(weight, covar) as weight -- [hivemall v0.2 or later]
+from 
+ (select 
+     -- train_multiclass_scw(addBias(features),label) as (label,feature,weight) -- [hivemall v0.1]
+     train_multiclass_scw(addBias(features),label) as (label,feature,weight,covar) -- [hivemall v0.2 or later]
+  from 
+     news20mc_train_x3
+ ) t 
+group by label, feature;
+```
+
+## prediction
+```sql
+create or replace view news20mc_scw_predict1 
+as
+select 
+  rowid, 
+  m.col0 as score, 
+  m.col1 as label
+from (
+select
+   rowid, 
+   maxrow(score, label) as m
+from (
+  select
+    t.rowid,
+    m.label,
+    sum(m.weight * t.value) as score
+  from 
+    news20mc_test_exploded t LEFT OUTER JOIN
+    news20mc_scw_model1 m ON (t.feature = m.feature)
+  group by
+    t.rowid, m.label
+) t1
+group by rowid
+) t2;
+```
+
+## evaluation
+```sql
+create or replace view news20mc_scw_submit1 as
+select 
+  t.label as actual, 
+  pd.label as predicted
+from 
+  news20mc_test t JOIN news20mc_scw_predict1 pd 
+    on (t.rowid = pd.rowid);
+```
+
+```
+select count(1)/3993 from news20mc_scw_submit1 
+where actual == predicted;
+```
+
+> 0.8314550463310794
+
+## Cleaning
+
+```sql
+drop table news20mc_scw_model1;
+drop table news20mc_scw_predict1;
+drop view news20mc_scw_submit1;
+```
+
+---
+#[SCW2]
+
+## training
+```sql
+drop table news20mc_scw2_model1;
+create table news20mc_scw2_model1 as
+select 
+ label, 
+ cast(feature as int) as feature,
+ -- voted_avg(weight) as weight -- [hivemall v0.1]
+ argmin_kld(weight, covar) as weight -- [hivemall v0.2 or later]
+from 
+ (select 
+     -- train_multiclass_scw2(addBias(features),label) as (label,feature,weight) -- [hivemall v0.1]
+     train_multiclass_scw2(addBias(features),label) as (label,feature,weight,covar) -- [hivemall v0.2 or later]
+  from 
+     news20mc_train_x3
+ ) t 
+group by label, feature;
+```
+
+## prediction
+```sql
+create or replace view news20mc_scw2_predict1 
+as
+select 
+  rowid, 
+  m.col0 as score, 
+  m.col1 as label
+from (
+select
+   rowid, 
+   maxrow(score, label) as m
+from (
+  select
+    t.rowid,
+    m.label,
+    sum(m.weight * t.value) as score
+  from 
+    news20mc_test_exploded t LEFT OUTER JOIN
+    news20mc_scw2_model1 m ON (t.feature = m.feature)
+  group by
+    t.rowid, m.label
+) t1
+group by rowid
+) t2;
+```
+
+## evaluation
+```sql
+create or replace view news20mc_scw2_submit1 as
+select 
+  t.label as actual, 
+  pd.label as predicted
+from 
+  news20mc_test t JOIN news20mc_scw2_predict1 pd 
+    on (t.rowid = pd.rowid);
+```
+
+```
+select count(1)/3993 from news20mc_scw2_submit1 
+where actual == predicted;
+```
+
+> 0.8482344102178813
+
+## Cleaning
+
+```sql
+drop table news20mc_scw2_model1;
+drop table news20mc_scw2_predict1;
+drop view news20mc_scw2_submit1;
+```
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/370e2aa3/docs/gitbook/pig/.gitkeep
----------------------------------------------------------------------
diff --git a/docs/gitbook/pig/.gitkeep b/docs/gitbook/pig/.gitkeep
new file mode 100644
index 0000000..e69de29

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/370e2aa3/docs/gitbook/recommend/cf.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/recommend/cf.md b/docs/gitbook/recommend/cf.md
new file mode 100644
index 0000000..e69de29


[22/50] [abbrv] incubator-hivemall git commit: Updated license headers

Posted by my...@apache.org.
Updated license headers

Project: http://git-wip-us.apache.org/repos/asf/incubator-hivemall/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-hivemall/commit/d806bf45
Tree: http://git-wip-us.apache.org/repos/asf/incubator-hivemall/tree/d806bf45
Diff: http://git-wip-us.apache.org/repos/asf/incubator-hivemall/diff/d806bf45

Branch: refs/heads/master
Commit: d806bf45b37bc9946a536c7d6ed38d34b208d45d
Parents: f443c06
Author: myui <yu...@gmail.com>
Authored: Fri Oct 28 21:05:46 2016 +0900
Committer: myui <yu...@gmail.com>
Committed: Fri Oct 28 21:12:59 2016 +0900

----------------------------------------------------------------------
 DISCLAIMER                                      |  1 +
 bin/format_header.sh                            | 26 +++++++++++++
 core/pom.xml                                    | 24 ++----------
 .../main/java/hivemall/HivemallConstants.java   | 28 +++++++-------
 .../main/java/hivemall/HivemallVersionUDF.java  | 28 +++++++-------
 .../src/main/java/hivemall/LearnerBaseUDTF.java | 28 +++++++-------
 core/src/main/java/hivemall/UDFWithOptions.java | 28 +++++++-------
 .../src/main/java/hivemall/UDTFWithOptions.java | 28 +++++++-------
 .../java/hivemall/anomaly/ChangeFinder1D.java   | 27 +++++++-------
 .../java/hivemall/anomaly/ChangeFinder2D.java   | 27 +++++++-------
 .../java/hivemall/anomaly/ChangeFinderUDF.java  | 27 +++++++-------
 core/src/main/java/hivemall/anomaly/SDAR1D.java | 27 +++++++-------
 core/src/main/java/hivemall/anomaly/SDAR2D.java | 27 +++++++-------
 .../hivemall/classifier/AROWClassifierUDTF.java | 28 +++++++-------
 .../hivemall/classifier/AdaGradRDAUDTF.java     | 28 +++++++-------
 .../classifier/BinaryOnlineClassifierUDTF.java  | 28 +++++++-------
 .../classifier/ConfidenceWeightedUDTF.java      | 28 +++++++-------
 .../classifier/PassiveAggressiveUDTF.java       | 28 +++++++-------
 .../hivemall/classifier/PerceptronUDTF.java     | 28 +++++++-------
 .../classifier/SoftConfideceWeightedUDTF.java   | 28 +++++++-------
 .../MulticlassAROWClassifierUDTF.java           | 28 +++++++-------
 .../MulticlassConfidenceWeightedUDTF.java       | 28 +++++++-------
 .../MulticlassOnlineClassifierUDTF.java         | 28 +++++++-------
 .../MulticlassPassiveAggressiveUDTF.java        | 28 +++++++-------
 .../multiclass/MulticlassPerceptronUDTF.java    | 28 +++++++-------
 .../MulticlassSoftConfidenceWeightedUDTF.java   | 28 +++++++-------
 .../java/hivemall/common/ConversionState.java   | 28 +++++++-------
 .../main/java/hivemall/common/EtaEstimator.java | 28 +++++++-------
 .../java/hivemall/common/LossFunctions.java     | 28 +++++++-------
 .../java/hivemall/common/OnlineVariance.java    | 28 +++++++-------
 .../hivemall/common/RandomizedAmplifier.java    | 28 +++++++-------
 .../java/hivemall/common/ReservoirSampler.java  | 28 +++++++-------
 .../LogisticRegressionDataGeneratorUDTF.java    | 28 +++++++-------
 .../hivemall/ensemble/ArgminKLDistanceUDAF.java | 28 +++++++-------
 .../main/java/hivemall/ensemble/MaxRowUDAF.java | 28 +++++++-------
 .../hivemall/ensemble/MaxValueLabelUDAF.java    | 28 +++++++-------
 .../hivemall/ensemble/bagging/VotedAvgUDAF.java | 28 +++++++-------
 .../ensemble/bagging/WeightVotedAvgUDAF.java    | 28 +++++++-------
 .../main/java/hivemall/evaluation/AUCUDAF.java  | 27 +++++++-------
 .../evaluation/BinaryResponsesMeasures.java     | 28 +++++++-------
 .../java/hivemall/evaluation/FMeasureUDAF.java  | 28 +++++++-------
 .../evaluation/GradedResponsesMeasures.java     | 28 +++++++-------
 .../evaluation/LogarithmicLossUDAF.java         | 27 +++++++-------
 .../main/java/hivemall/evaluation/MAPUDAF.java  | 27 +++++++-------
 .../main/java/hivemall/evaluation/MRRUDAF.java  | 27 +++++++-------
 .../evaluation/MeanAbsoluteErrorUDAF.java       | 28 +++++++-------
 .../evaluation/MeanSquaredErrorUDAF.java        | 28 +++++++-------
 .../main/java/hivemall/evaluation/NDCGUDAF.java | 27 +++++++-------
 .../java/hivemall/evaluation/PrecisionUDAF.java | 27 +++++++-------
 .../main/java/hivemall/evaluation/R2UDAF.java   | 28 +++++++-------
 .../java/hivemall/evaluation/RecallUDAF.java    | 27 +++++++-------
 .../evaluation/RootMeanSquaredErrorUDAF.java    | 28 +++++++-------
 core/src/main/java/hivemall/fm/Entry.java       | 28 +++++++-------
 .../main/java/hivemall/fm/FFMPredictUDF.java    | 28 +++++++-------
 .../java/hivemall/fm/FFMPredictionModel.java    | 28 +++++++-------
 .../hivemall/fm/FFMStringFeatureMapModel.java   | 28 +++++++-------
 .../src/main/java/hivemall/fm/FMArrayModel.java | 28 +++++++-------
 .../java/hivemall/fm/FMHyperParameters.java     | 27 +++++++-------
 .../java/hivemall/fm/FMIntFeatureMapModel.java  | 28 +++++++-------
 .../java/hivemall/fm/FMPredictGenericUDAF.java  | 28 +++++++-------
 .../hivemall/fm/FMStringFeatureMapModel.java    | 28 +++++++-------
 .../hivemall/fm/FactorizationMachineModel.java  | 28 +++++++-------
 .../hivemall/fm/FactorizationMachineUDTF.java   | 28 +++++++-------
 core/src/main/java/hivemall/fm/Feature.java     | 28 +++++++-------
 .../fm/FieldAwareFactorizationMachineModel.java | 28 +++++++-------
 .../fm/FieldAwareFactorizationMachineUDTF.java  | 28 +++++++-------
 core/src/main/java/hivemall/fm/IntFeature.java  | 28 +++++++-------
 .../main/java/hivemall/fm/StringFeature.java    | 28 +++++++-------
 .../main/java/hivemall/ftvec/AddBiasUDF.java    | 28 +++++++-------
 .../java/hivemall/ftvec/AddFeatureIndexUDF.java | 28 +++++++-------
 .../java/hivemall/ftvec/ExtractFeatureUDF.java  | 28 +++++++-------
 .../java/hivemall/ftvec/ExtractWeightUDF.java   | 28 +++++++-------
 .../java/hivemall/ftvec/FeatureIndexUDF.java    | 28 +++++++-------
 .../main/java/hivemall/ftvec/FeatureUDF.java    | 27 +++++++-------
 .../java/hivemall/ftvec/SortByFeatureUDF.java   | 28 +++++++-------
 .../hivemall/ftvec/amplify/AmplifierUDTF.java   | 28 +++++++-------
 .../ftvec/amplify/RandomAmplifierUDTF.java      | 28 +++++++-------
 .../hivemall/ftvec/binning/BuildBinsUDAF.java   | 27 +++++++-------
 .../ftvec/binning/FeatureBinningUDF.java        | 27 +++++++-------
 .../ftvec/binning/NumericHistogram.java         | 27 +++++++-------
 .../ftvec/conv/ConvertToDenseModelUDAF.java     | 28 +++++++-------
 .../ftvec/conv/QuantifyColumnsUDTF.java         | 28 +++++++-------
 .../hivemall/ftvec/conv/ToDenseFeaturesUDF.java | 27 +++++++-------
 .../ftvec/conv/ToSparseFeaturesUDF.java         | 27 +++++++-------
 .../ftvec/hashing/ArrayHashValuesUDF.java       | 28 +++++++-------
 .../hashing/ArrayPrefixedHashValuesUDF.java     | 28 +++++++-------
 .../ftvec/hashing/FeatureHashingUDF.java        | 27 +++++++-------
 .../hivemall/ftvec/hashing/MurmurHash3UDF.java  | 28 +++++++-------
 .../java/hivemall/ftvec/hashing/Sha1UDF.java    | 28 +++++++-------
 .../ftvec/pairing/PolynomialFeaturesUDF.java    | 28 +++++++-------
 .../ftvec/pairing/PoweredFeaturesUDF.java       | 28 +++++++-------
 .../hivemall/ftvec/ranking/BprSamplingUDTF.java | 28 +++++++-------
 .../ftvec/ranking/ItemPairsSamplingUDTF.java    | 28 +++++++-------
 .../ranking/PerEventPositiveOnlyFeedback.java   | 28 +++++++-------
 .../ftvec/ranking/PopulateNotInUDTF.java        | 28 +++++++-------
 .../ftvec/ranking/PositiveOnlyFeedback.java     | 28 +++++++-------
 .../ftvec/scaling/L2NormalizationUDF.java       | 28 +++++++-------
 .../java/hivemall/ftvec/scaling/RescaleUDF.java | 28 +++++++-------
 .../java/hivemall/ftvec/scaling/ZScoreUDF.java  | 28 +++++++-------
 .../hivemall/ftvec/text/TermFrequencyUDAF.java  | 28 +++++++-------
 .../hivemall/ftvec/trans/BinarizeLabelUDTF.java | 27 +++++++-------
 .../ftvec/trans/CategoricalFeaturesUDF.java     | 28 +++++++-------
 .../hivemall/ftvec/trans/FFMFeaturesUDF.java    | 28 +++++++-------
 .../hivemall/ftvec/trans/IndexedFeatures.java   | 27 +++++++-------
 .../ftvec/trans/OnehotEncodingUDAF.java         | 27 +++++++-------
 .../ftvec/trans/QuantifiedFeaturesUDTF.java     | 28 +++++++-------
 .../ftvec/trans/QuantitativeFeaturesUDF.java    | 28 +++++++-------
 .../ftvec/trans/VectorizeFeaturesUDF.java       | 28 +++++++-------
 .../knn/distance/AngularDistanceUDF.java        | 28 +++++++-------
 .../knn/distance/CosineDistanceUDF.java         | 28 +++++++-------
 .../knn/distance/EuclidDistanceUDF.java         | 28 +++++++-------
 .../knn/distance/HammingDistanceUDF.java        | 28 +++++++-------
 .../knn/distance/JaccardDistanceUDF.java        | 28 +++++++-------
 .../hivemall/knn/distance/KLDivergenceUDF.java  | 28 +++++++-------
 .../knn/distance/ManhattanDistanceUDF.java      | 28 +++++++-------
 .../knn/distance/MinkowskiDistanceUDF.java      | 28 +++++++-------
 .../java/hivemall/knn/distance/PopcountUDF.java | 28 +++++++-------
 .../main/java/hivemall/knn/lsh/MinHashUDTF.java | 28 +++++++-------
 .../java/hivemall/knn/lsh/MinHashesUDF.java     | 28 +++++++-------
 .../java/hivemall/knn/lsh/bBitMinHashUDF.java   | 28 +++++++-------
 .../knn/similarity/AngularSimilarityUDF.java    | 18 +++++++++
 .../knn/similarity/CosineSimilarityUDF.java     | 28 +++++++-------
 .../knn/similarity/Distance2SimilarityUDF.java  | 28 +++++++-------
 .../knn/similarity/EuclidSimilarity.java        | 28 +++++++-------
 .../knn/similarity/JaccardIndexUDF.java         | 28 +++++++-------
 .../java/hivemall/mf/BPRMFPredictionUDF.java    | 28 +++++++-------
 .../hivemall/mf/BPRMatrixFactorizationUDTF.java | 28 +++++++-------
 .../main/java/hivemall/mf/FactorizedModel.java  | 28 +++++++-------
 .../main/java/hivemall/mf/MFPredictionUDF.java  | 28 +++++++-------
 .../mf/MatrixFactorizationAdaGradUDTF.java      | 28 +++++++-------
 .../hivemall/mf/MatrixFactorizationSGDUDTF.java | 28 +++++++-------
 .../mf/OnlineMatrixFactorizationUDTF.java       | 28 +++++++-------
 core/src/main/java/hivemall/mf/Rating.java      | 28 +++++++-------
 .../main/java/hivemall/mf/RatingInitilizer.java | 28 +++++++-------
 core/src/main/java/hivemall/mix/MixEnv.java     | 28 +++++++-------
 core/src/main/java/hivemall/mix/MixMessage.java | 28 +++++++-------
 .../java/hivemall/mix/MixMessageDecoder.java    | 28 +++++++-------
 .../java/hivemall/mix/MixMessageEncoder.java    | 28 +++++++-------
 core/src/main/java/hivemall/mix/MixedModel.java | 28 +++++++-------
 .../src/main/java/hivemall/mix/MixedWeight.java | 28 +++++++-------
 core/src/main/java/hivemall/mix/NodeInfo.java   | 28 +++++++-------
 .../java/hivemall/mix/client/MixClient.java     | 28 +++++++-------
 .../hivemall/mix/client/MixClientHandler.java   | 28 +++++++-------
 .../mix/client/MixClientInitializer.java        | 28 +++++++-------
 .../hivemall/mix/client/MixRequestRouter.java   | 28 +++++++-------
 .../hivemall/model/AbstractPredictionModel.java | 28 +++++++-------
 .../main/java/hivemall/model/DenseModel.java    | 28 +++++++-------
 .../main/java/hivemall/model/FeatureValue.java  | 28 +++++++-------
 .../main/java/hivemall/model/IWeightValue.java  | 28 +++++++-------
 core/src/main/java/hivemall/model/Margin.java   | 28 +++++++-------
 .../java/hivemall/model/ModelUpdateHandler.java | 28 +++++++-------
 .../java/hivemall/model/PredictionModel.java    | 28 +++++++-------
 .../java/hivemall/model/PredictionResult.java   | 28 +++++++-------
 .../model/SpaceEfficientDenseModel.java         | 28 +++++++-------
 .../main/java/hivemall/model/SparseModel.java   | 28 +++++++-------
 .../model/SynchronizedModelWrapper.java         | 28 +++++++-------
 .../main/java/hivemall/model/WeightValue.java   | 28 +++++++-------
 .../hivemall/model/WeightValueWithClock.java    | 28 +++++++-------
 .../hivemall/regression/AROWRegressionUDTF.java | 28 +++++++-------
 .../java/hivemall/regression/AdaDeltaUDTF.java  | 28 +++++++-------
 .../java/hivemall/regression/AdaGradUDTF.java   | 28 +++++++-------
 .../java/hivemall/regression/LogressUDTF.java   | 28 +++++++-------
 .../PassiveAggressiveRegressionUDTF.java        | 28 +++++++-------
 .../hivemall/regression/RegressionBaseUDTF.java | 28 +++++++-------
 .../src/main/java/hivemall/smile/ModelType.java | 27 +++++++-------
 .../smile/classification/DecisionTree.java      | 27 +++++++-------
 .../GradientTreeBoostingClassifierUDTF.java     | 28 +++++++-------
 .../RandomForestClassifierUDTF.java             | 28 +++++++-------
 .../java/hivemall/smile/data/Attribute.java     | 28 +++++++-------
 .../regression/RandomForestRegressionUDTF.java  | 28 +++++++-------
 .../smile/regression/RegressionTree.java        | 27 +++++++-------
 .../smile/tools/GuessAttributesUDF.java         | 28 +++++++-------
 .../smile/tools/RandomForestEnsembleUDAF.java   | 28 +++++++-------
 .../hivemall/smile/tools/TreePredictUDF.java    | 27 +++++++-------
 .../hivemall/smile/utils/SmileExtUtils.java     | 27 +++++++-------
 .../hivemall/smile/utils/SmileTaskExecutor.java | 27 +++++++-------
 .../main/java/hivemall/smile/vm/Operation.java  | 27 +++++++-------
 .../java/hivemall/smile/vm/StackMachine.java    | 27 +++++++-------
 .../hivemall/smile/vm/VMRuntimeException.java   | 27 +++++++-------
 .../java/hivemall/tools/ConvertLabelUDF.java    | 28 +++++++-------
 .../main/java/hivemall/tools/EachTopKUDTF.java  | 28 +++++++-------
 .../java/hivemall/tools/GenerateSeriesUDTF.java | 28 +++++++-------
 .../java/hivemall/tools/RankSequenceUDF.java    | 28 +++++++-------
 .../tools/array/AllocFloatArrayUDF.java         | 28 +++++++-------
 .../tools/array/ArrayAvgGenericUDAF.java        | 28 +++++++-------
 .../hivemall/tools/array/ArrayConcatUDF.java    | 28 +++++++-------
 .../hivemall/tools/array/ArrayIntersectUDF.java | 28 +++++++-------
 .../hivemall/tools/array/ArrayRemoveUDF.java    | 28 +++++++-------
 .../java/hivemall/tools/array/ArraySumUDAF.java | 28 +++++++-------
 .../hivemall/tools/array/CollectAllUDAF.java    | 28 +++++++-------
 .../tools/array/SortAndUniqArrayUDF.java        | 28 +++++++-------
 .../tools/array/SubarrayEndWithUDF.java         | 28 +++++++-------
 .../tools/array/SubarrayStartWithUDF.java       | 28 +++++++-------
 .../java/hivemall/tools/array/SubarrayUDF.java  | 28 +++++++-------
 .../hivemall/tools/array/ToStringArrayUDF.java  | 28 +++++++-------
 .../hivemall/tools/bits/BitsCollectUDAF.java    | 28 +++++++-------
 .../java/hivemall/tools/bits/BitsORUDF.java     | 28 +++++++-------
 .../java/hivemall/tools/bits/ToBitsUDF.java     | 18 +++++++++
 .../java/hivemall/tools/bits/UnBitsUDF.java     | 18 +++++++++
 .../hivemall/tools/compress/DeflateUDF.java     | 28 +++++++-------
 .../hivemall/tools/compress/InflateUDF.java     | 28 +++++++-------
 .../java/hivemall/tools/map/MapGetSumUDF.java   | 28 +++++++-------
 .../java/hivemall/tools/map/MapTailNUDF.java    | 28 +++++++-------
 .../main/java/hivemall/tools/map/UDAFToMap.java | 28 +++++++-------
 .../hivemall/tools/map/UDAFToOrderedMap.java    | 28 +++++++-------
 .../tools/mapred/DistributedCacheLookupUDF.java | 28 +++++++-------
 .../hivemall/tools/mapred/JobConfGetsUDF.java   | 28 +++++++-------
 .../java/hivemall/tools/mapred/JobIdUDF.java    | 28 +++++++-------
 .../java/hivemall/tools/mapred/RowIdUDF.java    | 28 +++++++-------
 .../java/hivemall/tools/mapred/TaskIdUDF.java   | 28 +++++++-------
 .../hivemall/tools/math/SigmoidGenericUDF.java  | 28 +++++++-------
 .../java/hivemall/tools/text/Base91UDF.java     | 27 +++++++-------
 .../tools/text/NormalizeUnicodeUDF.java         | 28 +++++++-------
 .../java/hivemall/tools/text/SplitWordsUDF.java | 28 +++++++-------
 .../java/hivemall/tools/text/StopwordUDF.java   | 28 +++++++-------
 .../java/hivemall/tools/text/TokenizeUDF.java   | 28 +++++++-------
 .../java/hivemall/tools/text/Unbase91UDF.java   | 27 +++++++-------
 .../main/java/hivemall/utils/HivemallUtils.java | 28 +++++++-------
 .../hivemall/utils/buffer/DynamicByteArray.java | 27 +++++++-------
 .../java/hivemall/utils/buffer/HeapBuffer.java  | 27 +++++++-------
 .../main/java/hivemall/utils/codec/Base91.java  | 27 +++++++-------
 .../hivemall/utils/codec/CompressionCodec.java  | 28 +++++++-------
 .../java/hivemall/utils/codec/DeflateCodec.java | 28 +++++++-------
 .../hivemall/utils/codec/VariableByteCodec.java | 27 +++++++-------
 .../hivemall/utils/codec/ZigZagLEB128Codec.java | 27 +++++++-------
 .../utils/collections/BoundedPriorityQueue.java | 28 +++++++-------
 .../utils/collections/DoubleArray3D.java        | 28 +++++++-------
 .../utils/collections/DoubleArrayList.java      | 27 +++++++-------
 .../utils/collections/DoubleRingBuffer.java     | 27 +++++++-------
 .../utils/collections/FixedIntArray.java        | 27 +++++++-------
 .../utils/collections/IMapIterator.java         | 28 +++++++-------
 .../hivemall/utils/collections/IndexedSet.java  | 28 +++++++-------
 .../collections/Int2FloatOpenHashTable.java     | 28 +++++++-------
 .../utils/collections/Int2IntOpenHashTable.java | 28 +++++++-------
 .../collections/Int2LongOpenHashTable.java      | 28 +++++++-------
 .../hivemall/utils/collections/IntArray.java    | 27 +++++++-------
 .../utils/collections/IntArrayList.java         | 27 +++++++-------
 .../utils/collections/IntOpenHashMap.java       | 28 +++++++-------
 .../utils/collections/IntOpenHashTable.java     | 28 +++++++-------
 .../java/hivemall/utils/collections/LRUMap.java | 28 +++++++-------
 .../hivemall/utils/collections/OpenHashMap.java | 28 +++++++-------
 .../utils/collections/OpenHashTable.java        | 28 +++++++-------
 .../hivemall/utils/collections/RingBuffer.java  | 27 +++++++-------
 .../utils/collections/SparseIntArray.java       | 27 +++++++-------
 .../utils/concurrent/ExecutorFactory.java       | 27 +++++++-------
 .../utils/concurrent/NamedThreadFactory.java    | 27 +++++++-------
 .../utils/datetime/DateTimeFormatter.java       | 28 +++++++-------
 .../java/hivemall/utils/datetime/StopWatch.java | 28 +++++++-------
 .../java/hivemall/utils/hadoop/HadoopUtils.java | 28 +++++++-------
 .../java/hivemall/utils/hadoop/HiveUtils.java   | 28 +++++++-------
 .../main/java/hivemall/utils/hadoop/Text2.java  | 27 +++++++-------
 .../main/java/hivemall/utils/hadoop/Text3.java  | 27 +++++++-------
 .../hivemall/utils/hadoop/WritableUtils.java    | 28 +++++++-------
 .../hivemall/utils/hashing/HashFunction.java    | 28 +++++++-------
 .../utils/hashing/HashFunctionFactory.java      | 28 +++++++-------
 .../hivemall/utils/hashing/MurmurHash3.java     | 28 +++++++-------
 .../utils/hashing/MurmurHash3Function.java      | 28 +++++++-------
 .../hivemall/utils/io/ASCII85InputStream.java   | 27 +++++++-------
 .../hivemall/utils/io/ASCII85OutputStream.java  | 27 +++++++-------
 .../hivemall/utils/io/Base91InputStream.java    | 27 +++++++-------
 .../hivemall/utils/io/Base91OutputStream.java   | 27 +++++++-------
 .../utils/io/CompressionStreamFactory.java      | 28 +++++++-------
 .../hivemall/utils/io/DeflaterOutputStream.java | 27 +++++++-------
 .../utils/io/FastByteArrayInputStream.java      | 28 +++++++-------
 .../utils/io/FastByteArrayOutputStream.java     | 28 +++++++-------
 .../io/FastMultiByteArrayOutputStream.java      | 28 +++++++-------
 .../main/java/hivemall/utils/io/FileUtils.java  | 28 +++++++-------
 .../utils/io/FinishableOutputStream.java        | 28 +++++++-------
 .../utils/io/FinishableOutputStreamAdapter.java | 27 +++++++-------
 .../main/java/hivemall/utils/io/IOUtils.java    | 28 +++++++-------
 .../main/java/hivemall/utils/io/NIOUtils.java   | 28 +++++++-------
 .../java/hivemall/utils/io/NioFixedSegment.java | 28 +++++++-------
 .../main/java/hivemall/utils/io/NioSegment.java | 28 +++++++-------
 .../hivemall/utils/io/NioStatefullSegment.java  | 28 +++++++-------
 .../main/java/hivemall/utils/io/Segments.java   | 28 +++++++-------
 .../java/hivemall/utils/lang/ArrayUtils.java    | 28 +++++++-------
 .../main/java/hivemall/utils/lang/BitUtils.java | 28 +++++++-------
 .../hivemall/utils/lang/CommandLineUtils.java   | 28 +++++++-------
 .../main/java/hivemall/utils/lang/Copyable.java | 28 +++++++-------
 .../main/java/hivemall/utils/lang/Counter.java  | 28 +++++++-------
 .../hivemall/utils/lang/FloatAccumulator.java   | 28 +++++++-------
 .../java/hivemall/utils/lang/HalfFloat.java     | 28 +++++++-------
 .../java/hivemall/utils/lang/Identifier.java    | 28 +++++++-------
 .../java/hivemall/utils/lang/NumberUtils.java   | 28 +++++++-------
 .../java/hivemall/utils/lang/ObjectUtils.java   | 28 +++++++-------
 .../java/hivemall/utils/lang/Preconditions.java | 27 +++++++-------
 .../java/hivemall/utils/lang/Primitives.java    | 28 +++++++-------
 .../hivemall/utils/lang/PrivilegedAccessor.java | 28 +++++++-------
 .../java/hivemall/utils/lang/RandomUtils.java   | 27 +++++++-------
 .../main/java/hivemall/utils/lang/SizeOf.java   | 28 +++++++-------
 .../java/hivemall/utils/lang/StringUtils.java   | 28 +++++++-------
 .../java/hivemall/utils/lang/UnsafeUtils.java   | 27 +++++++-------
 .../utils/lang/mutable/MutableDouble.java       | 28 +++++++-------
 .../utils/lang/mutable/MutableFloat.java        | 28 +++++++-------
 .../hivemall/utils/lang/mutable/MutableInt.java | 28 +++++++-------
 .../utils/lang/mutable/MutableLong.java         | 28 +++++++-------
 .../src/main/java/hivemall/utils/lock/Lock.java | 28 +++++++-------
 .../main/java/hivemall/utils/lock/TTASLock.java | 28 +++++++-------
 .../java/hivemall/utils/math/MathUtils.java     | 28 +++++++-------
 .../java/hivemall/utils/math/MatrixUtils.java   | 27 +++++++-------
 .../main/java/hivemall/utils/math/Primes.java   | 28 +++++++-------
 .../java/hivemall/utils/math/StatsUtils.java    | 28 +++++++-------
 .../main/java/hivemall/utils/net/NetUtils.java  | 28 +++++++-------
 .../hive/ql/exec/MapredContextAccessor.java     | 28 +++++++-------
 .../hivemall/anomaly/ChangeFinder1DTest.java    | 27 +++++++-------
 .../hivemall/anomaly/ChangeFinder2DTest.java    | 27 +++++++-------
 .../classifier/PassiveAggressiveUDTFTest.java   | 28 +++++++-------
 .../hivemall/classifier/PerceptronUDTFTest.java | 28 +++++++-------
 .../hivemall/common/OnlineVarianceTest.java     | 28 +++++++-------
 .../common/RandomizedAmplifierTest.java         | 28 +++++++-------
 .../evaluation/BinaryResponsesMeasuresTest.java | 27 +++++++-------
 .../evaluation/GradedResponsesMeasuresTest.java | 27 +++++++-------
 .../test/java/hivemall/fm/ArrayModelTest.java   | 28 +++++++-------
 .../hivemall/fm/FFMPredictionModelTest.java     | 27 +++++++-------
 .../fm/FactorizationMachineUDTFTest.java        | 18 +++++++++
 core/src/test/java/hivemall/fm/FeatureTest.java | 27 +++++++-------
 .../FieldAwareFactorizationMachineUDTFTest.java | 28 +++++++-------
 .../hivemall/fm/IntFeatureMapModelTest.java     | 28 +++++++-------
 .../hivemall/fm/StringFeatureMapModelTest.java  | 28 +++++++-------
 .../hivemall/ftvec/ExtractWeightUDFTest.java    | 27 +++++++-------
 .../java/hivemall/ftvec/FeatureUDFTest.java     | 27 +++++++-------
 .../ftvec/hashing/FeatureHashingUDFTest.java    | 28 +++++++-------
 .../ftvec/hashing/MurmurHash3UDFTest.java       | 28 +++++++-------
 .../hivemall/ftvec/hashing/Sha1UDFTest.java     | 28 +++++++-------
 .../pairing/PolynomialFeaturesUDFTest.java      | 28 +++++++-------
 .../ftvec/pairing/PoweredFeaturesUDFTest.java   | 28 +++++++-------
 .../ftvec/scaling/L2NormalizationUDFTest.java   | 28 +++++++-------
 .../ftvec/trans/TestBinarizeLabelUDTF.java      | 27 +++++++-------
 .../ftvec/trans/VectorizeFeaturesUDFTest.java   | 28 +++++++-------
 .../knn/distance/EuclidDistanceUDFTest.java     | 28 +++++++-------
 .../java/hivemall/knn/lsh/MinHashUDFTest.java   | 28 +++++++-------
 .../hivemall/knn/lsh/bBitMinHashUDFTest.java    | 28 +++++++-------
 .../knn/similarity/CosineSimilarityUDFTest.java | 28 +++++++-------
 .../mf/BPRMatrixFactorizationUDTFTest.java      | 28 +++++++-------
 .../mf/MatrixFactorizationAdaGradUDTFTest.java  | 28 +++++++-------
 .../mf/MatrixFactorizationSGDUDTFTest.java      | 28 +++++++-------
 .../mix/client/MixRequestRouterTest.java        | 28 +++++++-------
 .../java/hivemall/model/FeatureValueTest.java   | 28 +++++++-------
 .../model/SpaceEfficientDenseModelTest.java     | 28 +++++++-------
 .../hivemall/regression/AdaGradUDTFTest.java    | 28 +++++++-------
 .../smile/classification/DecisionTreeTest.java  | 28 +++++++-------
 .../RandomForestClassifierUDTFTest.java         | 28 +++++++-------
 .../smile/regression/RegressionTreeTest.java    | 28 +++++++-------
 .../smile/tools/TreePredictUDFTest.java         | 28 +++++++-------
 .../hivemall/smile/vm/StackMachineTest.java     | 28 +++++++-------
 .../tools/array/ToStringArrayUDFTest.java       | 28 +++++++-------
 .../java/hivemall/utils/ArrayUtilsTest.java     | 28 +++++++-------
 .../test/java/hivemall/utils/BitUtilsTest.java  | 28 +++++++-------
 .../hivemall/utils/buffer/HeapBufferTest.java   | 27 +++++++-------
 .../java/hivemall/utils/codec/Base91Test.java   | 27 +++++++-------
 .../hivemall/utils/codec/DeflateCodecTest.java  | 28 +++++++-------
 .../utils/codec/ZigZagLEB128CodecTest.java      | 18 +++++++++
 .../utils/collections/DoubleArray3DTest.java    | 27 +++++++-------
 .../utils/collections/DoubleRingBufferTest.java | 27 +++++++-------
 .../collections/Int2FloatOpenHashMapTest.java   | 28 +++++++-------
 .../collections/Int2LongOpenHashMapTest.java    | 28 +++++++-------
 .../utils/collections/IntOpenHashMapTest.java   | 28 +++++++-------
 .../utils/collections/IntOpenHashTableTest.java | 28 +++++++-------
 .../utils/collections/OpenHashMapTest.java      | 28 +++++++-------
 .../utils/collections/OpenHashTableTest.java    | 28 +++++++-------
 .../utils/collections/SparseIntArrayTest.java   | 18 +++++++++
 .../hivemall/utils/hadoop/HadoopUtilsTest.java  | 28 +++++++-------
 .../java/hivemall/utils/hadoop/Text2Test.java   | 28 +++++++-------
 .../hivemall/utils/hashing/MurmurHash3Test.java | 28 +++++++-------
 .../utils/io/Base91OutputStreamTest.java        | 27 +++++++-------
 .../java/hivemall/utils/lang/HalfFloatTest.java | 28 +++++++-------
 .../hivemall/utils/lang/NumberUtilsTest.java    | 28 +++++++-------
 .../hivemall/utils/math/MatrixUtilsTest.java    | 27 +++++++-------
 mixserv/pom.xml                                 | 24 ++----------
 .../hivemall/mix/metrics/MetricsRegistry.java   | 28 +++++++-------
 .../hivemall/mix/metrics/MixServerMetrics.java  | 28 +++++++-------
 .../mix/metrics/MixServerMetricsMBean.java      | 28 +++++++-------
 .../hivemall/mix/metrics/ThroughputCounter.java | 28 +++++++-------
 .../java/hivemall/mix/server/MixServer.java     | 28 +++++++-------
 .../hivemall/mix/server/MixServerHandler.java   | 28 +++++++-------
 .../mix/server/MixServerInitializer.java        | 28 +++++++-------
 .../hivemall/mix/store/PartialArgminKLD.java    | 28 +++++++-------
 .../java/hivemall/mix/store/PartialAverage.java | 28 +++++++-------
 .../java/hivemall/mix/store/PartialResult.java  | 28 +++++++-------
 .../java/hivemall/mix/store/SessionObject.java  | 28 +++++++-------
 .../java/hivemall/mix/store/SessionStore.java   | 28 +++++++-------
 .../mix/server/MixServerHandlerTest.java        | 28 +++++++-------
 .../java/hivemall/mix/server/MixServerTest.java | 28 +++++++-------
 .../hivemall/mix/server/PartialResultTest.java  | 28 +++++++-------
 .../java/hivemall/test/HivemallTestBase.java    | 28 +++++++-------
 nlp/pom.xml                                     | 24 ++----------
 .../hivemall/nlp/tokenizer/KuromojiUDF.java     | 28 +++++++-------
 .../hivemall/nlp/tokenizer/KuromojiUDFTest.java | 28 +++++++-------
 pom.xml                                         | 39 +++++++++++++++++++-
 resources/license-header.txt                    | 28 +++++++-------
 .../org/apache/spark/sql/hive/HiveShim.scala    | 27 +++++++-------
 spark/spark-1.6/pom.xml                         |  1 +
 .../hivemall/tools/RegressionDatagen.scala      | 27 +++++++-------
 .../apache/spark/sql/hive/GroupedDataEx.scala   | 27 +++++++-------
 .../org/apache/spark/sql/hive/HivemallOps.scala | 27 +++++++-------
 .../apache/spark/sql/hive/HivemallUtils.scala   | 27 +++++++-------
 .../hivemall/mix/server/MixServerSuite.scala    | 27 +++++++-------
 .../hivemall/tools/RegressionDatagenSuite.scala | 27 +++++++-------
 .../scala/org/apache/spark/SparkFunSuite.scala  | 27 +++++++-------
 .../ml/feature/HivemallLabeledPointSuite.scala  | 27 +++++++-------
 .../scala/org/apache/spark/sql/QueryTest.scala  | 27 +++++++-------
 .../spark/sql/catalyst/plans/PlanTest.scala     | 27 +++++++-------
 .../apache/spark/sql/hive/HiveUdfSuite.scala    | 27 +++++++-------
 .../spark/sql/hive/HivemallOpsSuite.scala       | 27 +++++++-------
 .../spark/sql/hive/ModelMixingSuite.scala       | 27 +++++++-------
 .../spark/streaming/HivemallOpsSuite.scala      | 27 +++++++-------
 .../apache/spark/test/HivemallQueryTest.scala   | 27 +++++++-------
 .../scala/org/apache/spark/test/TestUtils.scala | 27 +++++++-------
 .../org/apache/spark/sql/hive/HiveShim.scala    | 27 +++++++-------
 spark/spark-2.0/pom.xml                         |  1 +
 .../java/hivemall/xgboost/XGBoostOptions.scala  | 27 +++++++-------
 .../XGBoostBinaryClassifierUDTFWrapper.java     | 27 +++++++-------
 .../XGBoostMulticlassClassifierUDTFWrapper.java | 27 +++++++-------
 .../main/java/hivemall/xgboost/package.scala    | 29 +++++++--------
 .../XGBoostRegressionUDTFWrapper.java           | 27 +++++++-------
 .../hivemall/tools/RegressionDatagen.scala      | 27 +++++++-------
 .../sql/catalyst/expressions/EachTopK.scala     | 27 +++++++-------
 .../spark/sql/hive/HivemallGroupedDataset.scala | 27 +++++++-------
 .../org/apache/spark/sql/hive/HivemallOps.scala | 27 +++++++-------
 .../apache/spark/sql/hive/HivemallUtils.scala   | 27 +++++++-------
 .../sql/hive/source/XGBoostFileFormat.scala     | 27 +++++++-------
 .../hivemall/mix/server/MixServerSuite.scala    | 27 +++++++-------
 .../hivemall/tools/RegressionDatagenSuite.scala | 27 +++++++-------
 .../scala/org/apache/spark/SparkFunSuite.scala  | 27 +++++++-------
 .../ml/feature/HivemallLabeledPointSuite.scala  | 27 +++++++-------
 .../scala/org/apache/spark/sql/QueryTest.scala  | 27 +++++++-------
 .../spark/sql/catalyst/plans/PlanTest.scala     | 27 +++++++-------
 .../apache/spark/sql/hive/HiveUdfSuite.scala    | 27 +++++++-------
 .../spark/sql/hive/HivemallOpsSuite.scala       | 27 +++++++-------
 .../spark/sql/hive/ModelMixingSuite.scala       | 27 +++++++-------
 .../apache/spark/sql/hive/XGBoostSuite.scala    | 27 +++++++-------
 .../sql/hive/benchmark/MiscBenchmark.scala      | 27 +++++++-------
 .../spark/sql/hive/test/TestHiveSingleton.scala | 27 +++++++-------
 .../streaming/HivemallFeatureOpsSuite.scala     | 27 +++++++-------
 .../spark/test/HivemallFeatureQueryTest.scala   | 27 +++++++-------
 .../scala/org/apache/spark/test/TestUtils.scala | 27 +++++++-------
 .../org/apache/spark/test/VectorQueryTest.scala | 27 +++++++-------
 spark/spark-common/pom.xml                      |  1 +
 ...isticRegressionDataGeneratorUDTFWrapper.java | 27 +++++++-------
 .../java/hivemall/ftvec/AddBiasUDFWrapper.java  | 27 +++++++-------
 .../ftvec/AddFeatureIndexUDFWrapper.java        | 27 +++++++-------
 .../ftvec/ExtractFeatureUDFWrapper.java         | 27 +++++++-------
 .../hivemall/ftvec/ExtractWeightUDFWrapper.java | 27 +++++++-------
 .../hivemall/ftvec/SortByFeatureUDFWrapper.java | 27 +++++++-------
 .../scaling/L2NormalizationUDFWrapper.java      | 27 +++++++-------
 .../hivemall/knn/lsh/MinHashesUDFWrapper.java   | 27 +++++++-------
 .../hivemall/tools/mapred/RowIdUDFWrapper.java  | 27 +++++++-------
 .../main/scala/hivemall/HivemallException.scala | 27 +++++++-------
 .../spark/ml/feature/HivemallLabeledPoint.scala | 27 +++++++-------
 .../spark/streaming/HivemallStreamingOps.scala  | 27 +++++++-------
 xgboost/pom.xml                                 | 21 +----------
 .../java/hivemall/xgboost/NativeLibLoader.java  | 28 +++++++-------
 .../hivemall/xgboost/XGBoostPredictUDTF.java    | 28 +++++++-------
 .../main/java/hivemall/xgboost/XGBoostUDTF.java | 28 +++++++-------
 .../java/hivemall/xgboost/XGBoostUtils.java     | 28 +++++++-------
 .../XGBoostBinaryClassifierUDTF.java            | 28 +++++++-------
 .../XGBoostMulticlassClassifierUDTF.java        | 28 +++++++-------
 .../regression/XGBoostRegressionUDTF.java       | 28 +++++++-------
 .../tools/XGBoostMulticlassPredictUDTF.java     | 28 +++++++-------
 .../xgboost/tools/XGBoostPredictUDTF.java       | 28 +++++++-------
 460 files changed, 6405 insertions(+), 6167 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/DISCLAIMER
----------------------------------------------------------------------
diff --git a/DISCLAIMER b/DISCLAIMER
new file mode 100644
index 0000000..ef89091
--- /dev/null
+++ b/DISCLAIMER
@@ -0,0 +1 @@
+Apache Hivemall is an effort undergoing incubation at The Apache Software Foundation (ASF), sponsored by the Apache Incubator PMC. Incubation is required of all newly accepted projects until a further review indicates that the infrastructure, communications, and decision making process have stabilized in a manner consistent with other successful ASF projects. While incubation status is not necessarily a reflection of the completeness or stability of the code, it does indicate that the project has yet to be fully endorsed by the ASF.

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/bin/format_header.sh
----------------------------------------------------------------------
diff --git a/bin/format_header.sh b/bin/format_header.sh
new file mode 100755
index 0000000..e02b1b5
--- /dev/null
+++ b/bin/format_header.sh
@@ -0,0 +1,26 @@
+#!/bin/sh
+
+if [ "$HIVEMALL_HOME" == "" ]; then
+  if [ -e ../bin/${0##*/} ]; then
+    HIVEMALL_HOME=".."
+  elif [ -e ./bin/${0##*/} ]; then
+    HIVEMALL_HOME="."
+  else
+    echo "env HIVEMALL_HOME not defined"
+    exit 1
+  fi
+fi
+
+cd $HIVEMALL_HOME
+HIVEMALL_HOME=`pwd`
+
+mvn license:format
+
+cd $HIVEMALL_HOME/spark/spark-common
+mvn license:format -P spark-2.0
+
+cd $HIVEMALL_HOME/spark/spark-1.6
+mvn license:format -P spark-1.6
+
+cd $HIVEMALL_HOME/spark/spark-2.0
+mvn license:format -P spark-2.0

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/pom.xml
----------------------------------------------------------------------
diff --git a/core/pom.xml b/core/pom.xml
index b9d2fbc..bddca2a 100644
--- a/core/pom.xml
+++ b/core/pom.xml
@@ -13,6 +13,10 @@
 	<name>Hivemall Core</name>
 	<packaging>jar</packaging>
 
+	<properties>
+		<main.basedir>${project.parent.basedir}</main.basedir>
+	</properties>
+
 	<dependencies>
 		<!-- provided scope -->
 		<dependency>
@@ -142,26 +146,6 @@
 		<finalName>${project.artifactId}-${project.version}</finalName>
 		<testOutputDirectory>target/test-classes</testOutputDirectory>
 		<plugins>
-			<plugin>
-				<groupId>com.mycila</groupId>
-				<artifactId>license-maven-plugin</artifactId>
-				<version>2.8</version>
-				<configuration>
-					<header>${project.parent.basedir}/resources/license-header.txt</header>
-					<properties>
-						<currentYear>${build.year}</currentYear>
-						<copyrightOwner>${project.organization.name}</copyrightOwner>
-					</properties>
-					<includes>
-						<include>src/main/**/*.java</include>
-						<include>src/test/**/*.java</include>
-					</includes>
-					<encoding>UTF-8</encoding>
-					<headerDefinitions>
-						<headerDefinition>${project.parent.basedir}/resources/header-definition.xml</headerDefinition>
-					</headerDefinitions>
-				</configuration>
-			</plugin>
 			<!-- hivemall-core-xx.jar -->
 			<plugin>
 				<groupId>org.apache.maven.plugins</groupId>

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/HivemallConstants.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/HivemallConstants.java b/core/src/main/java/hivemall/HivemallConstants.java
index d6de3b6..0eb9feb 100644
--- a/core/src/main/java/hivemall/HivemallConstants.java
+++ b/core/src/main/java/hivemall/HivemallConstants.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/HivemallVersionUDF.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/HivemallVersionUDF.java b/core/src/main/java/hivemall/HivemallVersionUDF.java
index f7293d1..c42701f 100644
--- a/core/src/main/java/hivemall/HivemallVersionUDF.java
+++ b/core/src/main/java/hivemall/HivemallVersionUDF.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/LearnerBaseUDTF.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/LearnerBaseUDTF.java b/core/src/main/java/hivemall/LearnerBaseUDTF.java
index 4518cce..f1ad99e 100644
--- a/core/src/main/java/hivemall/LearnerBaseUDTF.java
+++ b/core/src/main/java/hivemall/LearnerBaseUDTF.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/UDFWithOptions.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/UDFWithOptions.java b/core/src/main/java/hivemall/UDFWithOptions.java
index f4c6fb8..3aaf9ef 100644
--- a/core/src/main/java/hivemall/UDFWithOptions.java
+++ b/core/src/main/java/hivemall/UDFWithOptions.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/UDTFWithOptions.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/UDTFWithOptions.java b/core/src/main/java/hivemall/UDTFWithOptions.java
index 2546fc5..1556a4f 100644
--- a/core/src/main/java/hivemall/UDTFWithOptions.java
+++ b/core/src/main/java/hivemall/UDTFWithOptions.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/anomaly/ChangeFinder1D.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/anomaly/ChangeFinder1D.java b/core/src/main/java/hivemall/anomaly/ChangeFinder1D.java
index b2916c5..ce92be7 100644
--- a/core/src/main/java/hivemall/anomaly/ChangeFinder1D.java
+++ b/core/src/main/java/hivemall/anomaly/ChangeFinder1D.java
@@ -1,19 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.anomaly;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/anomaly/ChangeFinder2D.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/anomaly/ChangeFinder2D.java b/core/src/main/java/hivemall/anomaly/ChangeFinder2D.java
index 6eb3bf4..6ff6ea2 100644
--- a/core/src/main/java/hivemall/anomaly/ChangeFinder2D.java
+++ b/core/src/main/java/hivemall/anomaly/ChangeFinder2D.java
@@ -1,19 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.anomaly;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/anomaly/ChangeFinderUDF.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/anomaly/ChangeFinderUDF.java b/core/src/main/java/hivemall/anomaly/ChangeFinderUDF.java
index 250372b..7498255 100644
--- a/core/src/main/java/hivemall/anomaly/ChangeFinderUDF.java
+++ b/core/src/main/java/hivemall/anomaly/ChangeFinderUDF.java
@@ -1,19 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.anomaly;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/anomaly/SDAR1D.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/anomaly/SDAR1D.java b/core/src/main/java/hivemall/anomaly/SDAR1D.java
index e58b8a9..92185be 100644
--- a/core/src/main/java/hivemall/anomaly/SDAR1D.java
+++ b/core/src/main/java/hivemall/anomaly/SDAR1D.java
@@ -1,19 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.anomaly;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/anomaly/SDAR2D.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/anomaly/SDAR2D.java b/core/src/main/java/hivemall/anomaly/SDAR2D.java
index 6290c10..fbc9b1f 100644
--- a/core/src/main/java/hivemall/anomaly/SDAR2D.java
+++ b/core/src/main/java/hivemall/anomaly/SDAR2D.java
@@ -1,19 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.anomaly;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/classifier/AROWClassifierUDTF.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/classifier/AROWClassifierUDTF.java b/core/src/main/java/hivemall/classifier/AROWClassifierUDTF.java
index e5ef975..01c5554 100644
--- a/core/src/main/java/hivemall/classifier/AROWClassifierUDTF.java
+++ b/core/src/main/java/hivemall/classifier/AROWClassifierUDTF.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.classifier;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/classifier/AdaGradRDAUDTF.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/classifier/AdaGradRDAUDTF.java b/core/src/main/java/hivemall/classifier/AdaGradRDAUDTF.java
index 1351bca..a3e77db 100644
--- a/core/src/main/java/hivemall/classifier/AdaGradRDAUDTF.java
+++ b/core/src/main/java/hivemall/classifier/AdaGradRDAUDTF.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.classifier;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/classifier/BinaryOnlineClassifierUDTF.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/classifier/BinaryOnlineClassifierUDTF.java b/core/src/main/java/hivemall/classifier/BinaryOnlineClassifierUDTF.java
index 43a124d..c9274e4 100644
--- a/core/src/main/java/hivemall/classifier/BinaryOnlineClassifierUDTF.java
+++ b/core/src/main/java/hivemall/classifier/BinaryOnlineClassifierUDTF.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.classifier;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/classifier/ConfidenceWeightedUDTF.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/classifier/ConfidenceWeightedUDTF.java b/core/src/main/java/hivemall/classifier/ConfidenceWeightedUDTF.java
index f41be2c..5c268d2 100644
--- a/core/src/main/java/hivemall/classifier/ConfidenceWeightedUDTF.java
+++ b/core/src/main/java/hivemall/classifier/ConfidenceWeightedUDTF.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.classifier;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/classifier/PassiveAggressiveUDTF.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/classifier/PassiveAggressiveUDTF.java b/core/src/main/java/hivemall/classifier/PassiveAggressiveUDTF.java
index 0213dec..5ffda7b 100644
--- a/core/src/main/java/hivemall/classifier/PassiveAggressiveUDTF.java
+++ b/core/src/main/java/hivemall/classifier/PassiveAggressiveUDTF.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.classifier;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/classifier/PerceptronUDTF.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/classifier/PerceptronUDTF.java b/core/src/main/java/hivemall/classifier/PerceptronUDTF.java
index a04b8d7..58d9fd8 100644
--- a/core/src/main/java/hivemall/classifier/PerceptronUDTF.java
+++ b/core/src/main/java/hivemall/classifier/PerceptronUDTF.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.classifier;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/classifier/SoftConfideceWeightedUDTF.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/classifier/SoftConfideceWeightedUDTF.java b/core/src/main/java/hivemall/classifier/SoftConfideceWeightedUDTF.java
index c7f675c..e95d3ba 100644
--- a/core/src/main/java/hivemall/classifier/SoftConfideceWeightedUDTF.java
+++ b/core/src/main/java/hivemall/classifier/SoftConfideceWeightedUDTF.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.classifier;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/classifier/multiclass/MulticlassAROWClassifierUDTF.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/classifier/multiclass/MulticlassAROWClassifierUDTF.java b/core/src/main/java/hivemall/classifier/multiclass/MulticlassAROWClassifierUDTF.java
index f1e3e32..1387803 100644
--- a/core/src/main/java/hivemall/classifier/multiclass/MulticlassAROWClassifierUDTF.java
+++ b/core/src/main/java/hivemall/classifier/multiclass/MulticlassAROWClassifierUDTF.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.classifier.multiclass;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/classifier/multiclass/MulticlassConfidenceWeightedUDTF.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/classifier/multiclass/MulticlassConfidenceWeightedUDTF.java b/core/src/main/java/hivemall/classifier/multiclass/MulticlassConfidenceWeightedUDTF.java
index 4cb34b4..033c9b8 100644
--- a/core/src/main/java/hivemall/classifier/multiclass/MulticlassConfidenceWeightedUDTF.java
+++ b/core/src/main/java/hivemall/classifier/multiclass/MulticlassConfidenceWeightedUDTF.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.classifier.multiclass;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/classifier/multiclass/MulticlassOnlineClassifierUDTF.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/classifier/multiclass/MulticlassOnlineClassifierUDTF.java b/core/src/main/java/hivemall/classifier/multiclass/MulticlassOnlineClassifierUDTF.java
index aa50055..8ae949f 100644
--- a/core/src/main/java/hivemall/classifier/multiclass/MulticlassOnlineClassifierUDTF.java
+++ b/core/src/main/java/hivemall/classifier/multiclass/MulticlassOnlineClassifierUDTF.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.classifier.multiclass;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/classifier/multiclass/MulticlassPassiveAggressiveUDTF.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/classifier/multiclass/MulticlassPassiveAggressiveUDTF.java b/core/src/main/java/hivemall/classifier/multiclass/MulticlassPassiveAggressiveUDTF.java
index 716c704..d665c59 100644
--- a/core/src/main/java/hivemall/classifier/multiclass/MulticlassPassiveAggressiveUDTF.java
+++ b/core/src/main/java/hivemall/classifier/multiclass/MulticlassPassiveAggressiveUDTF.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.classifier.multiclass;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/classifier/multiclass/MulticlassPerceptronUDTF.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/classifier/multiclass/MulticlassPerceptronUDTF.java b/core/src/main/java/hivemall/classifier/multiclass/MulticlassPerceptronUDTF.java
index 2308e60..2430389 100644
--- a/core/src/main/java/hivemall/classifier/multiclass/MulticlassPerceptronUDTF.java
+++ b/core/src/main/java/hivemall/classifier/multiclass/MulticlassPerceptronUDTF.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.classifier.multiclass;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/classifier/multiclass/MulticlassSoftConfidenceWeightedUDTF.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/classifier/multiclass/MulticlassSoftConfidenceWeightedUDTF.java b/core/src/main/java/hivemall/classifier/multiclass/MulticlassSoftConfidenceWeightedUDTF.java
index 8fca907..b2e7a45 100644
--- a/core/src/main/java/hivemall/classifier/multiclass/MulticlassSoftConfidenceWeightedUDTF.java
+++ b/core/src/main/java/hivemall/classifier/multiclass/MulticlassSoftConfidenceWeightedUDTF.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.classifier.multiclass;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/common/ConversionState.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/common/ConversionState.java b/core/src/main/java/hivemall/common/ConversionState.java
index 24daec9..dd20662 100644
--- a/core/src/main/java/hivemall/common/ConversionState.java
+++ b/core/src/main/java/hivemall/common/ConversionState.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.common;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/common/EtaEstimator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/common/EtaEstimator.java b/core/src/main/java/hivemall/common/EtaEstimator.java
index 3287641..0bfb9dc 100644
--- a/core/src/main/java/hivemall/common/EtaEstimator.java
+++ b/core/src/main/java/hivemall/common/EtaEstimator.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.common;
 



[32/50] [abbrv] incubator-hivemall git commit: Removed unused classes

Posted by my...@apache.org.
Removed unused classes

Project: http://git-wip-us.apache.org/repos/asf/incubator-hivemall/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-hivemall/commit/8cef8a3b
Tree: http://git-wip-us.apache.org/repos/asf/incubator-hivemall/tree/8cef8a3b
Diff: http://git-wip-us.apache.org/repos/asf/incubator-hivemall/diff/8cef8a3b

Branch: refs/heads/master
Commit: 8cef8a3bb5b90ab972e9f735f76d9e2585b9f9ea
Parents: 0cf2a64
Author: myui <yu...@gmail.com>
Authored: Sat Nov 12 14:53:19 2016 +0900
Committer: myui <yu...@gmail.com>
Committed: Sat Nov 12 14:53:19 2016 +0900

----------------------------------------------------------------------
 .../hivemall/utils/io/ASCII85InputStream.java   | 230 -------------------
 .../hivemall/utils/io/ASCII85OutputStream.java  | 151 ------------
 2 files changed, 381 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/8cef8a3b/core/src/main/java/hivemall/utils/io/ASCII85InputStream.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/utils/io/ASCII85InputStream.java b/core/src/main/java/hivemall/utils/io/ASCII85InputStream.java
deleted file mode 100644
index bc0ca16..0000000
--- a/core/src/main/java/hivemall/utils/io/ASCII85InputStream.java
+++ /dev/null
@@ -1,230 +0,0 @@
-/*
- * 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.utils.io;
-
-import java.io.FilterInputStream;
-import java.io.IOException;
-import java.io.InputStream;
-
-/**
- * This class represents an ASCII85 stream. This class is based on the implementation in Apache
- * PDFBox.
- */
-public final class ASCII85InputStream extends FilterInputStream {
-
-    private static final byte TERMINATOR = '~';
-    private static final byte OFFSET = '!';
-    private static final byte NEWLINE = '\n';
-    private static final byte RETURN = '\r';
-    private static final byte SPACE = ' ';
-    private static final byte PADDING_U = 'u';
-    private static final byte Z = 'z';
-
-    private int index;
-    private int n;
-    private boolean eof;
-
-    private final byte[] ascii;
-    private final byte[] decoded;
-
-    /**
-     * Constructor.
-     *
-     * @param is The input stream to actually read from.
-     */
-    public ASCII85InputStream(InputStream is) {
-        super(is);
-        index = 0;
-        n = 0;
-        eof = false;
-        ascii = new byte[5];
-        decoded = new byte[4];
-    }
-
-    /**
-     * This will read the next byte from the stream.
-     *
-     * @return The next byte read from the stream.
-     * @throws IOException If there is an error reading from the wrapped stream.
-     */
-    @Override
-    public int read() throws IOException {
-        if (index < n) {
-            return decoded[index++] & 0xFF;
-        }
-        if (eof) {
-            return -1;
-        }
-
-        index = 0;
-        int k;
-        byte z;
-        do {
-            int zz = (byte) in.read();
-            if (zz == -1) {
-                eof = true;
-                return -1;
-            }
-            z = (byte) zz;
-        } while (z == NEWLINE || z == RETURN || z == SPACE);
-
-        if (z == TERMINATOR) {
-            eof = true;
-            n = 0;
-            return -1;
-        } else if (z == Z) {
-            decoded[0] = decoded[1] = decoded[2] = decoded[3] = 0;
-            n = 4;
-        } else {
-            ascii[0] = z; // may be EOF here....
-            for (k = 1; k < 5; ++k) {
-                do {
-                    int zz = (byte) in.read();
-                    if (zz == -1) {
-                        eof = true;
-                        return -1;
-                    }
-                    z = (byte) zz;
-                } while (z == NEWLINE || z == RETURN || z == SPACE);
-                ascii[k] = z;
-                if (z == TERMINATOR) {
-                    // don't include ~ as padding byte
-                    ascii[k] = PADDING_U;
-                    break;
-                }
-            }
-            n = k - 1;
-            if (n == 0) {
-                eof = true;
-                return -1;
-            }
-            if (k < 5) {
-                for (++k; k < 5; ++k) {
-                    ascii[k] = PADDING_U;
-                }
-                eof = true;
-            }
-            // decode stream
-            long t = 0;
-            for (k = 0; k < 5; ++k) {
-                z = (byte) (ascii[k] - OFFSET);
-                if (z < 0 || z > 93) {
-                    throw new IOException("Invalid data in Ascii85 stream");
-                }
-                t = (t * 85L) + z;
-            }
-            for (k = 3; k >= 0; --k) {
-                decoded[k] = (byte) (t & 0xFFL);
-                t >>>= 8;
-            }
-        }
-        return decoded[index++] & 0xFF;
-    }
-
-    /**
-     * This will read a chunk of data.
-     *
-     * @param data The buffer to write data to.
-     * @param offset The offset into the data stream.
-     * @param len The number of byte to attempt to read.
-     *
-     * @return The number of bytes actually read.
-     *
-     * @throws IOException If there is an error reading data from the underlying stream.
-     */
-    @Override
-    public int read(final byte[] data, final int offset, final int len) throws IOException {
-        if (eof && index >= n) {
-            return -1;
-        }
-        for (int i = 0; i < len; i++) {
-            if (index < n) {
-                data[i + offset] = decoded[index++];
-            } else {
-                int t = read();
-                if (t == -1) {
-                    return i;
-                }
-                data[i + offset] = (byte) t;
-            }
-        }
-        return len;
-    }
-
-    /**
-     * This will close the underlying stream and release any resources.
-     *
-     * @throws IOException If there is an error closing the underlying stream.
-     */
-    @Override
-    public void close() throws IOException {
-        eof = true;
-        super.close();
-    }
-
-    /**
-     * non supported interface methods.
-     *
-     * @return False always.
-     */
-    @Override
-    public boolean markSupported() {
-        return false;
-    }
-
-    /**
-     * Unsupported.
-     *
-     * @param nValue ignored.
-     *
-     * @return Always zero.
-     */
-    @Override
-    public long skip(long nValue) {
-        return 0;
-    }
-
-    /**
-     * Unsupported.
-     *
-     * @return Always zero.
-     */
-    @Override
-    public int available() {
-        return 0;
-    }
-
-    /**
-     * Unsupported.
-     *
-     * @param readlimit ignored.
-     */
-    @Override
-    public void mark(int readlimit) {}
-
-    /**
-     * Unsupported.
-     *
-     * @throws IOException telling that this is an unsupported action.
-     */
-    @Override
-    public void reset() throws IOException {
-        throw new IOException("Reset is not supported");
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/8cef8a3b/core/src/main/java/hivemall/utils/io/ASCII85OutputStream.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/utils/io/ASCII85OutputStream.java b/core/src/main/java/hivemall/utils/io/ASCII85OutputStream.java
deleted file mode 100644
index fc7d440..0000000
--- a/core/src/main/java/hivemall/utils/io/ASCII85OutputStream.java
+++ /dev/null
@@ -1,151 +0,0 @@
-/*
- * 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.utils.io;
-
-import java.io.FilterOutputStream;
-import java.io.IOException;
-import java.io.OutputStream;
-
-/**
- * This class represents an ASCII85 output stream. This class is based on the implementation in
- * Apache PDFBox.
- */
-public final class ASCII85OutputStream extends FilterOutputStream {
-
-    private static final long a85p2 = 85L * 85L;
-    private static final long a85p3 = 85L * 85L * 85L;
-    private static final long a85p4 = 85L * 85L * 85L * 85L;
-    private static final byte TERMINATOR = '~';
-    private static final byte OFFSET = '!';
-    private static final byte Z = 'z';
-
-    private final byte[] indata;
-    private final byte[] encoded;
-
-    private int count;
-    private boolean flushed;
-
-    /**
-     * Constructor.
-     *
-     * @param out The output stream to write to.
-     */
-    public ASCII85OutputStream(OutputStream out) {
-        super(out);
-        indata = new byte[4];
-        encoded = new byte[5];
-        count = 0;
-        flushed = true;
-    }
-
-    /**
-     * This will transform the next four ascii bytes.
-     */
-    private void transformASCII85() {
-        long word = ((((indata[0] << 8) | (indata[1] & 0xFF)) << 16) | ((indata[2] & 0xFF) << 8) | (indata[3] & 0xFF)) & 0xFFFFFFFFL;
-
-        if (word == 0) {
-            encoded[0] = Z;
-            encoded[1] = 0;
-            return;
-        }
-
-        long x = word / a85p4;
-        encoded[0] = (byte) (x + OFFSET);
-        word -= x * a85p4;
-
-        x = word / a85p3;
-        encoded[1] = (byte) (x + OFFSET);
-        word -= x * a85p3;
-
-        x = word / a85p2;
-        encoded[2] = (byte) (x + OFFSET);
-        word -= x * a85p2;
-
-        x = word / 85L;
-        encoded[3] = (byte) (x + OFFSET);
-
-        encoded[4] = (byte) ((word % 85L) + OFFSET);
-    }
-
-    /**
-     * This will write a single byte.
-     *
-     * @param b The byte to write.
-     *
-     * @throws IOException If there is an error writing to the stream.
-     */
-    @Override
-    public void write(int b) throws IOException {
-        flushed = false;
-        indata[count++] = (byte) b;
-        if (count < 4) {
-            return;
-        }
-        transformASCII85();
-        for (int i = 0; i < 5; i++) {
-            if (encoded[i] == 0) {
-                break;
-            }
-            out.write(encoded[i]);
-        }
-        count = 0;
-    }
-
-    /**
-     * This will flush the data to the stream.
-     *
-     * @throws IOException If there is an error writing the data to the stream.
-     */
-    @Override
-    public void flush() throws IOException {
-        if (flushed) {
-            return;
-        }
-        if (count > 0) {
-            for (int i = count; i < 4; i++) {
-                indata[i] = 0;
-            }
-            transformASCII85();
-            if (encoded[0] == Z) {
-                for (int i = 0; i < 5; i++) {
-                    encoded[i] = OFFSET;// expand 'z',
-                }
-            }
-            for (int i = 0; i < count + 1; i++) {
-                out.write(encoded[i]);
-            }
-        }
-        out.write(TERMINATOR);
-        count = 0;
-        flushed = true;
-        super.flush();
-    }
-
-    /**
-     * This will close the stream.
-     *
-     * @throws IOException If there is an error closing the wrapped stream.
-     */
-    @Override
-    public void close() throws IOException {
-        flush();
-        super.close();
-    }
-}


[45/50] [abbrv] incubator-hivemall git commit: [HIVEMALL-4] Implement contribution guideline (#382)

Posted by my...@apache.org.
[HIVEMALL-4] Implement contribution guideline (#382)



Project: http://git-wip-us.apache.org/repos/asf/incubator-hivemall/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-hivemall/commit/da55b54d
Tree: http://git-wip-us.apache.org/repos/asf/incubator-hivemall/tree/da55b54d
Diff: http://git-wip-us.apache.org/repos/asf/incubator-hivemall/diff/da55b54d

Branch: refs/heads/master
Commit: da55b54dba648ee3576f2ce2dd20b0791a79e092
Parents: a71bbb7
Author: Kai Sasaki <le...@me.com>
Authored: Sun Nov 20 04:23:02 2016 +0900
Committer: Makoto YUI <yu...@gmail.com>
Committed: Sun Nov 20 04:23:02 2016 +0900

----------------------------------------------------------------------
 src/site/markdown/contributing.md | 96 +++++++++++++++++++++++++++++++++-
 1 file changed, 95 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/da55b54d/src/site/markdown/contributing.md
----------------------------------------------------------------------
diff --git a/src/site/markdown/contributing.md b/src/site/markdown/contributing.md
index 803d5f5..1b91e8f 100644
--- a/src/site/markdown/contributing.md
+++ b/src/site/markdown/contributing.md
@@ -35,6 +35,100 @@ $ mvn clean package
 
 # Contribution guideline
 
-to appear ...
+This guide documents the best way to make various types of contribution to Apache Hivemall, 
+including what is required before submitting a code change.
+
+Contributing to Hivemall doesn't just mean writing code. Helping new users on the [mailing list](/mail-lists.html), 
+testing releases, and improving documentation are also welcome. In fact, proposing significant code changes usually 
+requires first gaining experience and credibility within the community by helping in other ways. This is also a guide 
+to becoming an effective contributor. So, this guide organizes contributions in order that they should probably be 
+considered by new contributors who intend to get involved long-term. Build some track record of helping others, 
+rather than just open pull requests.
+
+## Preparing to contribute code changes
+
+Before proceeding, contributors should evaluate if the proposed change is likely to be relevant, new and actionable:
+
+* Is it clear that code must change? Proposing a [JIRA](https://issues.apache.org/jira/browse/HIVEMALL) and pull request is appropriate only when a clear problem or 
+change has been identified. If simply having trouble using Hivemall, use the [mailing lists](/mail-lists.html) first, 
+rather than consider filing a JIRA or proposing a change. When in doubt, email user@hivemall.incubator.apache.org first 
+about the possible change
+* Search the user@hivemall.incubator.apache.org and dev@hivemall.incubator.apache.org mailing list archives for related 
+discussions. Often, the problem has been discussed before, with a resolution that doesn't require a code change, or recording 
+what kinds of changes will not be accepted as a resolution.
+* Search JIRA for existing issues: https://issues.apache.org/jira/browse/HIVEMALL . Please search by typing keyword in search box.
+
+**When you contribute code, you affirm that the contribution is your original work and that you license the work to the project 
+under the project's open source license. Whether or not you state this explicitly, by submitting any copyrighted material via 
+pull request, email, or other means you agree to license the material under the project's open source license and warrant that 
+you have the legal authority to do so.**
+
+### JIRA
+
+Generally, Hivemall uses JIRA to track logical issues, including bugs and improvements, and uses Github pull requests to 
+manage the review and merge of specific code changes. That is, JIRAs are used to describe what should be fixed or changed, 
+and high-level approaches, and pull requests describe how to implement that change in the project's source code. 
+For example, major design decisions are discussed in JIRA.
+
+1. Find the existing [Hivemall](https://issues.apache.org/jira/browse/HIVEMALL) JIRA that the change pertains to.
+    * Do not create a new JIRA if creating a change to address an existing issue in JIRA; add to the existing discussion 
+and work instead 
+    * Look for existing pull requests that are linked from the JIRA, to understand if someone is already working on the JIRA 
+2. If the change is new, then it usually needs a new JIRA. However, trivial changes, where the what should change is virtually the same as the how it should change do not require a JIRA. Example: "Fix typos in Foo scaladoc"
+3. If required, create a new JIRA:
+    * Provide a descriptive Title. "Update web UI" or "Problem in scheduler" is not sufficient. "Inject the Hivemall aggregate functionality in RelationalGroupedDataset" is good.
+    * Write a detailed Description. For bug reports, this should ideally include a short reproduction of the problem. For new features, it may include a design document.
+    * Set required fields:
+        * *Issue Type*. Generally, Bug, Improvement and Test are the only types used in Hivemall.
+        * *Affects Version*. For Bugs, assign at least one version that is known to exhibit the problem or need the change
+        * Do not set the following fields:
+            * *Fix Version*. This is assigned by committers only when resolved.
+            * *Target Version*. This is assigned by committers to indicate a PR has been accepted for possible fix by the target version.
+4. If the change is a large change, consider inviting discussion on the issue at dev@hivemall.incubator.apache.org first before proceeding to implement the change.
+
+### Pull Request
+1. Fork the Github repository at http://github.com/apache/incubator-hivemall if you haven't already
+2. Clone your fork, create a new branch, push commits to the branch.
+3. Consider whether documentation or tests need to be added or updated as part of the change, and add them as needed.
+4. Run all tests with `mvn test` to verify that the code still compiles, passes tests, and passes style checks.
+5. Open a pull request against the master branch of [apache/incubator-hivemall](https://github.com/apache/incubator-hivemall). (Only in special cases would the PR be opened against other branches.)
+    * The PR title should be of the form [HIVEMALL-xxxx]  Title, where HIVEMALL-xxxx is the relevant JIRA number, Title may be the JIRA's title or a more specific title describing the PR itself.
+    * If the pull request is still a work in progress, and so is not ready to be merged, but needs to be pushed to Github to facilitate review, then add [WIP] head of the title.
+    * Consider identifying committers or other contributors who have worked on the code being changed. Find the file(s) in Github and click "Blame" to see a line-by-line annotation of who changed the code last. You can add @username in the PR description to ping them immediately.
+    * Please state that the contribution is your original work and that you license the work to the project under the project's open source license.
+    * The related JIRA, if any, will be marked as "In Progress" and your pull request will automatically be linked to it. There is no need to be the Assignee of the JIRA to work on it, though you are welcome to comment that you have begun work.
+6. The Jenkins automatic pull request builder will test your changes
+7. After about some minutes, Jenkins will post the results of the test to the pull request, along with a link to the full results on Jenkins.
+8. Watch for the results, and investigate and fix failures promptly
+    * Fixes can simply be pushed to the same branch from which you opened your pull request
+    * Jenkins will automatically re-test when new commits are pushed
+    * If the tests failed for reasons unrelated to the change (e.g. Jenkins outage), then a committer can request a re-test with "Jenkins, retest this please". Ask if you need a test restarted.
+ 
+## The Review Process
+* Other reviewers, including committers, may comment on the changes and suggest modifications. Changes can be added by simply pushing more commits to the same branch. 
+* Lively, polite, rapid technical debate is encouraged from everyone in the community. The outcome may be a rejection of the entire change. 
+* Reviewers can indicate that a change looks suitable for merging with a comment such as: "I think this patch looks good". Hivemall uses the LGTM convention for indicating 
+the strongest level of technical sign-off on a patch: simply comment with the word "LGTM". It specifically means: 
+"I've looked at this thoroughly and take as much ownership as if I wrote the patch myself". If you comment LGTM you 
+will be expected to help with bugs or follow-up issues on the patch. Consistent, judicious use of LGTMs is a great 
+way to gain credibility as a reviewer with the broader community. 
+* Sometimes, other changes will be merged which conflict 
+with your pull request's changes. The PR can't be merged until the conflict is resolved. This can be resolved with 
+"git fetch origin" followed by "git merge origin/master" and resolving the conflicts by hand, then pushing the result 
+to your branch. 
+* Try to be responsive to the discussion rather than let days pass between replies
+
+## Closing your pull request / JIRA
+
+* If a change is accepted, it will be merged and the pull request will automatically be closed, along with the associated JIRA if any
+    * Note that in the rare case you are asked to open a pull request against a branch besides master, that you will actually have to close the pull request manually
+    * The JIRA will be Assigned to the primary contributor to the change as a way of giving credit. If the JIRA isn't closed and/or Assigned promptly, comment on the JIRA.
+* If your pull request is ultimately rejected, please close it promptly
+* If a pull request has gotten little or no attention, consider improving the description or the change itself and ping likely reviewers again after a few days. Consider proposing a change that's easier to include, like a smaller and/or less invasive change.
+* If it has been reviewed but not taken up after weeks, after soliciting review from the most relevant reviewers, or, has met with neutral reactions, the outcome may be considered a "soft no". It is helpful to withdraw and close the PR in this case.
+* If a pull request is closed because it is deemed not the right approach to resolve a JIRA, then leave the JIRA open. However if the review makes it clear that the issue identified in the JIRA is not going to be resolved by any pull request (not a problem, won't fix) then also resolve the JIRA.
+
+
 
 [Here](http://www.apache.org/foundation/getinvolved.html) is a general guide for contributing to Apache Project.
+


[47/50] [abbrv] incubator-hivemall git commit: Update NOTICE to list up source codes borrowed from other porjects (#384)

Posted by my...@apache.org.
Update NOTICE to list up source codes borrowed from other porjects (#384)



Project: http://git-wip-us.apache.org/repos/asf/incubator-hivemall/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-hivemall/commit/effe4614
Tree: http://git-wip-us.apache.org/repos/asf/incubator-hivemall/tree/effe4614
Diff: http://git-wip-us.apache.org/repos/asf/incubator-hivemall/diff/effe4614

Branch: refs/heads/master
Commit: effe461446f2af84931901c112d542b8d2e69cfd
Parents: f9f9799
Author: Takeshi Yamamuro <li...@gmail.com>
Authored: Mon Nov 21 14:28:34 2016 +0900
Committer: Makoto YUI <yu...@gmail.com>
Committed: Mon Nov 21 14:28:34 2016 +0900

----------------------------------------------------------------------
 NOTICE | 11 ++++++++++-
 1 file changed, 10 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/effe4614/NOTICE
----------------------------------------------------------------------
diff --git a/NOTICE b/NOTICE
index 2f22827..b252238 100644
--- a/NOTICE
+++ b/NOTICE
@@ -23,7 +23,7 @@ o hivemall/core/src/main/java/hivemall/utils/codec/Base91.java
 
     Copyright (c) 2000-2006 Joachim Henke
 
-    https://github.com/bwaldvogel/base91    
+    https://github.com/bwaldvogel/base91
     Licensed under the BSD 3-Clause License
 
 o hivemall/core/src/main/java/hivemall/utils/collections/OpenHashMap.java
@@ -32,3 +32,12 @@ o hivemall/core/src/main/java/hivemall/utils/collections/OpenHashMap.java
 
     https://github.com/slipperyseal/atomicobjects/
     Licensed under the Apache License, Version 2.0
+
+o hivemall/spark/spark-1.6/extra-src/hive/src/main/scala/org/apache/spark/sql/hive/HiveShim.scala
+  hivemall/spark/spark-2.0/extra-src/hive/src/main/scala/org/apache/spark/sql/hive/HiveShim.scala
+  hivemall/spark/spark-2.0/src/test/scala/org/apache/spark/sql/QueryTest.scala
+  hivemall/spark/spark-2.0/src/test/scala/org/apache/spark/sql/catalyst/plans/PlanTest.scala
+  hivemall/spark/spark-2.0/src/test/scala/org/apache/spark/sql/hive/test/TestHiveSingleton.scala
+
+    https://github.com/apache/spark
+    Licensed under the Apache License, Version 2.0


[11/50] [abbrv] incubator-hivemall git commit: Updated license headers

Posted by my...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/spark/spark-common/src/main/scala/hivemall/HivemallException.scala
----------------------------------------------------------------------
diff --git a/spark/spark-common/src/main/scala/hivemall/HivemallException.scala b/spark/spark-common/src/main/scala/hivemall/HivemallException.scala
index 169bd6b..53f6756 100644
--- a/spark/spark-common/src/main/scala/hivemall/HivemallException.scala
+++ b/spark/spark-common/src/main/scala/hivemall/HivemallException.scala
@@ -1,20 +1,21 @@
 /*
- * 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
+ * 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
+ *   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.
+ * 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
 
 class HivemallException(message: String, cause: Throwable)

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/spark/spark-common/src/main/scala/org/apache/spark/ml/feature/HivemallLabeledPoint.scala
----------------------------------------------------------------------
diff --git a/spark/spark-common/src/main/scala/org/apache/spark/ml/feature/HivemallLabeledPoint.scala b/spark/spark-common/src/main/scala/org/apache/spark/ml/feature/HivemallLabeledPoint.scala
index 3dd9d20..3fb2d18 100644
--- a/spark/spark-common/src/main/scala/org/apache/spark/ml/feature/HivemallLabeledPoint.scala
+++ b/spark/spark-common/src/main/scala/org/apache/spark/ml/feature/HivemallLabeledPoint.scala
@@ -1,20 +1,21 @@
 /*
- * 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
+ * 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
+ *   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.
+ * 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 org.apache.spark.ml.feature
 
 import java.util.StringTokenizer

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/spark/spark-common/src/main/scala/org/apache/spark/streaming/HivemallStreamingOps.scala
----------------------------------------------------------------------
diff --git a/spark/spark-common/src/main/scala/org/apache/spark/streaming/HivemallStreamingOps.scala b/spark/spark-common/src/main/scala/org/apache/spark/streaming/HivemallStreamingOps.scala
index 72d6ca7..a6bbb4b 100644
--- a/spark/spark-common/src/main/scala/org/apache/spark/streaming/HivemallStreamingOps.scala
+++ b/spark/spark-common/src/main/scala/org/apache/spark/streaming/HivemallStreamingOps.scala
@@ -1,20 +1,21 @@
 /*
- * 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
+ * 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
+ *   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.
+ * 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 org.apache.spark.streaming
 
 import scala.reflect.ClassTag

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/xgboost/pom.xml
----------------------------------------------------------------------
diff --git a/xgboost/pom.xml b/xgboost/pom.xml
index da30566..677fbbd 100644
--- a/xgboost/pom.xml
+++ b/xgboost/pom.xml
@@ -16,6 +16,7 @@
 	<properties>
 		<xgboost.version>0.60</xgboost.version>
 		<xgboost4j.version>0.10</xgboost4j.version>
+		<main.basedir>${project.parent.basedir}</main.basedir>
 	</properties>
 
 	<dependencies>
@@ -97,26 +98,6 @@
 		<finalName>${project.artifactId}-${project.version}</finalName>
 		<testOutputDirectory>target/test-classes</testOutputDirectory>
 		<plugins>
-			<plugin>
-				<groupId>com.mycila</groupId>
-				<artifactId>license-maven-plugin</artifactId>
-				<version>2.8</version>
-				<configuration>
-					<header>${project.parent.basedir}/resources/license-header.txt</header>
-					<properties>
-						<currentYear>${build.year}</currentYear>
-						<copyrightOwner>${project.organization.name}</copyrightOwner>
-					</properties>
-					<includes>
-						<include>src/main/**/*.java</include>
-						<include>src/test/**/*.java</include>
-					</includes>
-					<encoding>UTF-8</encoding>
-					<headerDefinitions>
-						<headerDefinition>${project.parent.basedir}/resources/header-definition.xml</headerDefinition>
-					</headerDefinitions>
-				</configuration>
-			</plugin>
 			<!-- TODO: This is hacky, so we'll replace this with another better way in a future -->
 			<plugin>
 				<groupId>org.apache.maven.plugins</groupId>

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/xgboost/src/main/java/hivemall/xgboost/NativeLibLoader.java
----------------------------------------------------------------------
diff --git a/xgboost/src/main/java/hivemall/xgboost/NativeLibLoader.java b/xgboost/src/main/java/hivemall/xgboost/NativeLibLoader.java
index d4505ac..82347bf 100644
--- a/xgboost/src/main/java/hivemall/xgboost/NativeLibLoader.java
+++ b/xgboost/src/main/java/hivemall/xgboost/NativeLibLoader.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.xgboost;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/xgboost/src/main/java/hivemall/xgboost/XGBoostPredictUDTF.java
----------------------------------------------------------------------
diff --git a/xgboost/src/main/java/hivemall/xgboost/XGBoostPredictUDTF.java b/xgboost/src/main/java/hivemall/xgboost/XGBoostPredictUDTF.java
index a0ba0a9..33568c7 100644
--- a/xgboost/src/main/java/hivemall/xgboost/XGBoostPredictUDTF.java
+++ b/xgboost/src/main/java/hivemall/xgboost/XGBoostPredictUDTF.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.xgboost;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/xgboost/src/main/java/hivemall/xgboost/XGBoostUDTF.java
----------------------------------------------------------------------
diff --git a/xgboost/src/main/java/hivemall/xgboost/XGBoostUDTF.java b/xgboost/src/main/java/hivemall/xgboost/XGBoostUDTF.java
index 147ab88..b269549 100644
--- a/xgboost/src/main/java/hivemall/xgboost/XGBoostUDTF.java
+++ b/xgboost/src/main/java/hivemall/xgboost/XGBoostUDTF.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.xgboost;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/xgboost/src/main/java/hivemall/xgboost/XGBoostUtils.java
----------------------------------------------------------------------
diff --git a/xgboost/src/main/java/hivemall/xgboost/XGBoostUtils.java b/xgboost/src/main/java/hivemall/xgboost/XGBoostUtils.java
index 8f66f09..9705f94 100644
--- a/xgboost/src/main/java/hivemall/xgboost/XGBoostUtils.java
+++ b/xgboost/src/main/java/hivemall/xgboost/XGBoostUtils.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.xgboost;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/xgboost/src/main/java/hivemall/xgboost/classification/XGBoostBinaryClassifierUDTF.java
----------------------------------------------------------------------
diff --git a/xgboost/src/main/java/hivemall/xgboost/classification/XGBoostBinaryClassifierUDTF.java b/xgboost/src/main/java/hivemall/xgboost/classification/XGBoostBinaryClassifierUDTF.java
index d07ccde..7e135ec 100644
--- a/xgboost/src/main/java/hivemall/xgboost/classification/XGBoostBinaryClassifierUDTF.java
+++ b/xgboost/src/main/java/hivemall/xgboost/classification/XGBoostBinaryClassifierUDTF.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.xgboost.classification;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/xgboost/src/main/java/hivemall/xgboost/classification/XGBoostMulticlassClassifierUDTF.java
----------------------------------------------------------------------
diff --git a/xgboost/src/main/java/hivemall/xgboost/classification/XGBoostMulticlassClassifierUDTF.java b/xgboost/src/main/java/hivemall/xgboost/classification/XGBoostMulticlassClassifierUDTF.java
index 59c58ad..171e4bc 100644
--- a/xgboost/src/main/java/hivemall/xgboost/classification/XGBoostMulticlassClassifierUDTF.java
+++ b/xgboost/src/main/java/hivemall/xgboost/classification/XGBoostMulticlassClassifierUDTF.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.xgboost.classification;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/xgboost/src/main/java/hivemall/xgboost/regression/XGBoostRegressionUDTF.java
----------------------------------------------------------------------
diff --git a/xgboost/src/main/java/hivemall/xgboost/regression/XGBoostRegressionUDTF.java b/xgboost/src/main/java/hivemall/xgboost/regression/XGBoostRegressionUDTF.java
index 84a2553..d00b430 100644
--- a/xgboost/src/main/java/hivemall/xgboost/regression/XGBoostRegressionUDTF.java
+++ b/xgboost/src/main/java/hivemall/xgboost/regression/XGBoostRegressionUDTF.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.xgboost.regression;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/xgboost/src/main/java/hivemall/xgboost/tools/XGBoostMulticlassPredictUDTF.java
----------------------------------------------------------------------
diff --git a/xgboost/src/main/java/hivemall/xgboost/tools/XGBoostMulticlassPredictUDTF.java b/xgboost/src/main/java/hivemall/xgboost/tools/XGBoostMulticlassPredictUDTF.java
index 8becb3e..6ceb17e 100644
--- a/xgboost/src/main/java/hivemall/xgboost/tools/XGBoostMulticlassPredictUDTF.java
+++ b/xgboost/src/main/java/hivemall/xgboost/tools/XGBoostMulticlassPredictUDTF.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.xgboost.tools;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/xgboost/src/main/java/hivemall/xgboost/tools/XGBoostPredictUDTF.java
----------------------------------------------------------------------
diff --git a/xgboost/src/main/java/hivemall/xgboost/tools/XGBoostPredictUDTF.java b/xgboost/src/main/java/hivemall/xgboost/tools/XGBoostPredictUDTF.java
index eab0bc9..4510206 100644
--- a/xgboost/src/main/java/hivemall/xgboost/tools/XGBoostPredictUDTF.java
+++ b/xgboost/src/main/java/hivemall/xgboost/tools/XGBoostPredictUDTF.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.xgboost.tools;
 


[34/50] [abbrv] incubator-hivemall git commit: Removed an unnecessary annotation

Posted by my...@apache.org.
Removed an unnecessary annotation

Project: http://git-wip-us.apache.org/repos/asf/incubator-hivemall/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-hivemall/commit/d433a0a3
Tree: http://git-wip-us.apache.org/repos/asf/incubator-hivemall/tree/d433a0a3
Diff: http://git-wip-us.apache.org/repos/asf/incubator-hivemall/diff/d433a0a3

Branch: refs/heads/master
Commit: d433a0a329f264801779877dfa78c2149a098702
Parents: 2f8e44a
Author: myui <yu...@gmail.com>
Authored: Sat Nov 12 14:54:23 2016 +0900
Committer: myui <yu...@gmail.com>
Committed: Sat Nov 12 14:54:23 2016 +0900

----------------------------------------------------------------------
 core/src/main/java/hivemall/ftvec/binning/BuildBinsUDAF.java | 1 -
 1 file changed, 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d433a0a3/core/src/main/java/hivemall/ftvec/binning/BuildBinsUDAF.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/ftvec/binning/BuildBinsUDAF.java b/core/src/main/java/hivemall/ftvec/binning/BuildBinsUDAF.java
index 508d37c..995414d 100644
--- a/core/src/main/java/hivemall/ftvec/binning/BuildBinsUDAF.java
+++ b/core/src/main/java/hivemall/ftvec/binning/BuildBinsUDAF.java
@@ -228,7 +228,6 @@ public final class BuildBinsUDAF extends AbstractGenericUDAFResolver {
             }
         }
 
-        @SuppressWarnings("serial")
         @Override
         public Object terminatePartial(@SuppressWarnings("deprecation") AggregationBuffer agg)
                 throws HiveException {


[36/50] [abbrv] incubator-hivemall git commit: Merge pull request #380 from myui/jira/HIVEMALL-12

Posted by my...@apache.org.
Merge pull request #380 from myui/jira/HIVEMALL-12

jira/hivemall-12

Project: http://git-wip-us.apache.org/repos/asf/incubator-hivemall/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-hivemall/commit/28909548
Tree: http://git-wip-us.apache.org/repos/asf/incubator-hivemall/tree/28909548
Diff: http://git-wip-us.apache.org/repos/asf/incubator-hivemall/diff/28909548

Branch: refs/heads/master
Commit: 289095482f5a932472f1e86f30f7658a12b50753
Parents: 9a6d1c0 0268609
Author: Makoto YUI <yu...@gmail.com>
Authored: Sat Nov 12 15:04:24 2016 +0900
Committer: GitHub <no...@github.com>
Committed: Sat Nov 12 15:04:24 2016 +0900

----------------------------------------------------------------------
 DISCLAIMER                                      |   1 +
 NOTICE                                          |  36 ++-
 README.md                                       |  23 +-
 bin/build.sh                                    |  18 ++
 bin/build_site.sh                               |  18 ++
 bin/build_xgboost.sh                            |  27 +--
 bin/format_header.sh                            |  44 ++++
 bin/maven_central_release.sh                    |  18 ++
 bin/mixserv_cluster.sh                          |  27 +--
 bin/mixserv_daemon.sh                           |  27 +--
 bin/start_mixserv.sh                            |  27 +--
 bin/stop_mixserv.sh                             |  27 +--
 core/pom.xml                                    |  42 ++--
 .../main/java/hivemall/HivemallConstants.java   |  28 +--
 .../main/java/hivemall/HivemallVersionUDF.java  |  28 +--
 .../src/main/java/hivemall/LearnerBaseUDTF.java |  28 +--
 core/src/main/java/hivemall/UDFWithOptions.java |  28 +--
 .../src/main/java/hivemall/UDTFWithOptions.java |  28 +--
 .../java/hivemall/anomaly/ChangeFinder1D.java   |  27 +--
 .../java/hivemall/anomaly/ChangeFinder2D.java   |  27 +--
 .../java/hivemall/anomaly/ChangeFinderUDF.java  |  27 +--
 core/src/main/java/hivemall/anomaly/SDAR1D.java |  27 +--
 core/src/main/java/hivemall/anomaly/SDAR2D.java |  27 +--
 .../hivemall/classifier/AROWClassifierUDTF.java |  28 +--
 .../hivemall/classifier/AdaGradRDAUDTF.java     |  28 +--
 .../classifier/BinaryOnlineClassifierUDTF.java  |  28 +--
 .../classifier/ConfidenceWeightedUDTF.java      |  28 +--
 .../classifier/PassiveAggressiveUDTF.java       |  28 +--
 .../hivemall/classifier/PerceptronUDTF.java     |  28 +--
 .../classifier/SoftConfideceWeightedUDTF.java   |  28 +--
 .../MulticlassAROWClassifierUDTF.java           |  28 +--
 .../MulticlassConfidenceWeightedUDTF.java       |  28 +--
 .../MulticlassOnlineClassifierUDTF.java         |  28 +--
 .../MulticlassPassiveAggressiveUDTF.java        |  28 +--
 .../multiclass/MulticlassPerceptronUDTF.java    |  28 +--
 .../MulticlassSoftConfidenceWeightedUDTF.java   |  28 +--
 .../java/hivemall/common/ConversionState.java   |  28 +--
 .../main/java/hivemall/common/EtaEstimator.java |  28 +--
 .../java/hivemall/common/LossFunctions.java     |  28 +--
 .../java/hivemall/common/OnlineVariance.java    |  28 +--
 .../hivemall/common/RandomizedAmplifier.java    |  28 +--
 .../java/hivemall/common/ReservoirSampler.java  |  28 +--
 .../LogisticRegressionDataGeneratorUDTF.java    |  28 +--
 .../hivemall/ensemble/ArgminKLDistanceUDAF.java |  28 +--
 .../main/java/hivemall/ensemble/MaxRowUDAF.java |  43 ++--
 .../hivemall/ensemble/MaxValueLabelUDAF.java    |  28 +--
 .../hivemall/ensemble/bagging/VotedAvgUDAF.java |  28 +--
 .../ensemble/bagging/WeightVotedAvgUDAF.java    |  28 +--
 .../main/java/hivemall/evaluation/AUCUDAF.java  |  27 +--
 .../evaluation/BinaryResponsesMeasures.java     |  28 +--
 .../java/hivemall/evaluation/FMeasureUDAF.java  |  28 +--
 .../evaluation/GradedResponsesMeasures.java     |  28 +--
 .../evaluation/LogarithmicLossUDAF.java         |  27 +--
 .../main/java/hivemall/evaluation/MAPUDAF.java  |  27 +--
 .../main/java/hivemall/evaluation/MRRUDAF.java  |  27 +--
 .../evaluation/MeanAbsoluteErrorUDAF.java       |  28 +--
 .../evaluation/MeanSquaredErrorUDAF.java        |  28 +--
 .../main/java/hivemall/evaluation/NDCGUDAF.java |  27 +--
 .../java/hivemall/evaluation/PrecisionUDAF.java |  27 +--
 .../main/java/hivemall/evaluation/R2UDAF.java   |  28 +--
 .../java/hivemall/evaluation/RecallUDAF.java    |  27 +--
 .../evaluation/RootMeanSquaredErrorUDAF.java    |  28 +--
 core/src/main/java/hivemall/fm/Entry.java       |  28 +--
 .../main/java/hivemall/fm/FFMPredictUDF.java    |  28 +--
 .../java/hivemall/fm/FFMPredictionModel.java    |  28 +--
 .../hivemall/fm/FFMStringFeatureMapModel.java   |  28 +--
 .../src/main/java/hivemall/fm/FMArrayModel.java |  28 +--
 .../java/hivemall/fm/FMHyperParameters.java     |  27 +--
 .../java/hivemall/fm/FMIntFeatureMapModel.java  |  28 +--
 .../java/hivemall/fm/FMPredictGenericUDAF.java  |  28 +--
 .../hivemall/fm/FMStringFeatureMapModel.java    |  28 +--
 .../hivemall/fm/FactorizationMachineModel.java  |  28 +--
 .../hivemall/fm/FactorizationMachineUDTF.java   |  28 +--
 core/src/main/java/hivemall/fm/Feature.java     |  28 +--
 .../fm/FieldAwareFactorizationMachineModel.java |  28 +--
 .../fm/FieldAwareFactorizationMachineUDTF.java  |  28 +--
 core/src/main/java/hivemall/fm/IntFeature.java  |  28 +--
 .../main/java/hivemall/fm/StringFeature.java    |  28 +--
 .../main/java/hivemall/ftvec/AddBiasUDF.java    |  28 +--
 .../java/hivemall/ftvec/AddFeatureIndexUDF.java |  28 +--
 .../java/hivemall/ftvec/ExtractFeatureUDF.java  |  28 +--
 .../java/hivemall/ftvec/ExtractWeightUDF.java   |  28 +--
 .../java/hivemall/ftvec/FeatureIndexUDF.java    |  28 +--
 .../main/java/hivemall/ftvec/FeatureUDF.java    |  27 +--
 .../java/hivemall/ftvec/SortByFeatureUDF.java   |  28 +--
 .../hivemall/ftvec/amplify/AmplifierUDTF.java   |  28 +--
 .../ftvec/amplify/RandomAmplifierUDTF.java      |  28 +--
 .../hivemall/ftvec/binning/BuildBinsUDAF.java   |  28 +--
 .../ftvec/binning/FeatureBinningUDF.java        |  27 +--
 .../ftvec/binning/NumericHistogram.java         |  27 +--
 .../ftvec/conv/ConvertToDenseModelUDAF.java     |  28 +--
 .../ftvec/conv/QuantifyColumnsUDTF.java         |  28 +--
 .../hivemall/ftvec/conv/ToDenseFeaturesUDF.java |  27 +--
 .../ftvec/conv/ToSparseFeaturesUDF.java         |  27 +--
 .../ftvec/hashing/ArrayHashValuesUDF.java       |  28 +--
 .../hashing/ArrayPrefixedHashValuesUDF.java     |  28 +--
 .../ftvec/hashing/FeatureHashingUDF.java        |  27 +--
 .../hivemall/ftvec/hashing/MurmurHash3UDF.java  |  28 +--
 .../java/hivemall/ftvec/hashing/Sha1UDF.java    |  28 +--
 .../ftvec/pairing/PolynomialFeaturesUDF.java    |  28 +--
 .../ftvec/pairing/PoweredFeaturesUDF.java       |  28 +--
 .../hivemall/ftvec/ranking/BprSamplingUDTF.java |  28 +--
 .../ftvec/ranking/ItemPairsSamplingUDTF.java    |  28 +--
 .../ranking/PerEventPositiveOnlyFeedback.java   |  28 +--
 .../ftvec/ranking/PopulateNotInUDTF.java        |  28 +--
 .../ftvec/ranking/PositiveOnlyFeedback.java     |  28 +--
 .../ftvec/scaling/L2NormalizationUDF.java       |  28 +--
 .../java/hivemall/ftvec/scaling/RescaleUDF.java |  28 +--
 .../java/hivemall/ftvec/scaling/ZScoreUDF.java  |  28 +--
 .../hivemall/ftvec/text/TermFrequencyUDAF.java  |  28 +--
 .../hivemall/ftvec/trans/BinarizeLabelUDTF.java |  27 +--
 .../ftvec/trans/CategoricalFeaturesUDF.java     |  28 +--
 .../hivemall/ftvec/trans/FFMFeaturesUDF.java    |  28 +--
 .../hivemall/ftvec/trans/IndexedFeatures.java   |  27 +--
 .../ftvec/trans/OnehotEncodingUDAF.java         |  27 +--
 .../ftvec/trans/QuantifiedFeaturesUDTF.java     |  28 +--
 .../ftvec/trans/QuantitativeFeaturesUDF.java    |  28 +--
 .../ftvec/trans/VectorizeFeaturesUDF.java       |  28 +--
 .../knn/distance/AngularDistanceUDF.java        |  28 +--
 .../knn/distance/CosineDistanceUDF.java         |  28 +--
 .../knn/distance/EuclidDistanceUDF.java         |  28 +--
 .../knn/distance/HammingDistanceUDF.java        |  28 +--
 .../knn/distance/JaccardDistanceUDF.java        |  28 +--
 .../hivemall/knn/distance/KLDivergenceUDF.java  |  28 +--
 .../knn/distance/ManhattanDistanceUDF.java      |  28 +--
 .../knn/distance/MinkowskiDistanceUDF.java      |  28 +--
 .../java/hivemall/knn/distance/PopcountUDF.java |  28 +--
 .../main/java/hivemall/knn/lsh/MinHashUDTF.java |  28 +--
 .../java/hivemall/knn/lsh/MinHashesUDF.java     |  28 +--
 .../java/hivemall/knn/lsh/bBitMinHashUDF.java   |  28 +--
 .../knn/similarity/AngularSimilarityUDF.java    |  18 ++
 .../knn/similarity/CosineSimilarityUDF.java     |  28 +--
 .../knn/similarity/Distance2SimilarityUDF.java  |  28 +--
 .../knn/similarity/EuclidSimilarity.java        |  28 +--
 .../knn/similarity/JaccardIndexUDF.java         |  28 +--
 .../java/hivemall/mf/BPRMFPredictionUDF.java    |  28 +--
 .../hivemall/mf/BPRMatrixFactorizationUDTF.java |  28 +--
 .../main/java/hivemall/mf/FactorizedModel.java  |  28 +--
 .../main/java/hivemall/mf/MFPredictionUDF.java  |  28 +--
 .../mf/MatrixFactorizationAdaGradUDTF.java      |  28 +--
 .../hivemall/mf/MatrixFactorizationSGDUDTF.java |  28 +--
 .../mf/OnlineMatrixFactorizationUDTF.java       |  28 +--
 core/src/main/java/hivemall/mf/Rating.java      |  28 +--
 .../main/java/hivemall/mf/RatingInitilizer.java |  28 +--
 core/src/main/java/hivemall/mix/MixEnv.java     |  28 +--
 core/src/main/java/hivemall/mix/MixMessage.java |  28 +--
 .../java/hivemall/mix/MixMessageDecoder.java    |  28 +--
 .../java/hivemall/mix/MixMessageEncoder.java    |  28 +--
 core/src/main/java/hivemall/mix/MixedModel.java |  28 +--
 .../src/main/java/hivemall/mix/MixedWeight.java |  28 +--
 core/src/main/java/hivemall/mix/NodeInfo.java   |  28 +--
 .../java/hivemall/mix/client/MixClient.java     |  28 +--
 .../hivemall/mix/client/MixClientHandler.java   |  28 +--
 .../mix/client/MixClientInitializer.java        |  28 +--
 .../hivemall/mix/client/MixRequestRouter.java   |  28 +--
 .../hivemall/model/AbstractPredictionModel.java |  28 +--
 .../main/java/hivemall/model/DenseModel.java    |  28 +--
 .../main/java/hivemall/model/FeatureValue.java  |  28 +--
 .../main/java/hivemall/model/IWeightValue.java  |  28 +--
 core/src/main/java/hivemall/model/Margin.java   |  28 +--
 .../java/hivemall/model/ModelUpdateHandler.java |  28 +--
 .../java/hivemall/model/PredictionModel.java    |  28 +--
 .../java/hivemall/model/PredictionResult.java   |  28 +--
 .../model/SpaceEfficientDenseModel.java         |  28 +--
 .../main/java/hivemall/model/SparseModel.java   |  28 +--
 .../model/SynchronizedModelWrapper.java         |  28 +--
 .../main/java/hivemall/model/WeightValue.java   |  28 +--
 .../hivemall/model/WeightValueWithClock.java    |  28 +--
 .../hivemall/regression/AROWRegressionUDTF.java |  28 +--
 .../java/hivemall/regression/AdaDeltaUDTF.java  |  28 +--
 .../java/hivemall/regression/AdaGradUDTF.java   |  28 +--
 .../java/hivemall/regression/LogressUDTF.java   |  28 +--
 .../PassiveAggressiveRegressionUDTF.java        |  28 +--
 .../hivemall/regression/RegressionBaseUDTF.java |  28 +--
 .../src/main/java/hivemall/smile/ModelType.java |  27 +--
 .../smile/classification/DecisionTree.java      |  27 +--
 .../GradientTreeBoostingClassifierUDTF.java     |  28 +--
 .../RandomForestClassifierUDTF.java             |  28 +--
 .../java/hivemall/smile/data/Attribute.java     |  28 +--
 .../regression/RandomForestRegressionUDTF.java  |  28 +--
 .../smile/regression/RegressionTree.java        |  27 +--
 .../smile/tools/GuessAttributesUDF.java         |  28 +--
 .../smile/tools/RandomForestEnsembleUDAF.java   |  28 +--
 .../hivemall/smile/tools/TreePredictUDF.java    |  27 +--
 .../hivemall/smile/utils/SmileExtUtils.java     |  27 +--
 .../hivemall/smile/utils/SmileTaskExecutor.java |  27 +--
 .../main/java/hivemall/smile/vm/Operation.java  |  27 +--
 .../java/hivemall/smile/vm/StackMachine.java    |  27 +--
 .../hivemall/smile/vm/VMRuntimeException.java   |  27 +--
 .../java/hivemall/tools/ConvertLabelUDF.java    |  28 +--
 .../main/java/hivemall/tools/EachTopKUDTF.java  |  28 +--
 .../java/hivemall/tools/GenerateSeriesUDTF.java |  28 +--
 .../java/hivemall/tools/RankSequenceUDF.java    |  28 +--
 .../tools/array/AllocFloatArrayUDF.java         |  28 +--
 .../tools/array/ArrayAvgGenericUDAF.java        |  28 +--
 .../hivemall/tools/array/ArrayConcatUDF.java    |  28 +--
 .../hivemall/tools/array/ArrayIntersectUDF.java |  28 +--
 .../hivemall/tools/array/ArrayRemoveUDF.java    |  28 +--
 .../java/hivemall/tools/array/ArraySumUDAF.java |  28 +--
 .../hivemall/tools/array/CollectAllUDAF.java    |  28 +--
 .../tools/array/SortAndUniqArrayUDF.java        |  28 +--
 .../tools/array/SubarrayEndWithUDF.java         |  28 +--
 .../tools/array/SubarrayStartWithUDF.java       |  28 +--
 .../java/hivemall/tools/array/SubarrayUDF.java  |  28 +--
 .../hivemall/tools/array/ToStringArrayUDF.java  |  28 +--
 .../hivemall/tools/bits/BitsCollectUDAF.java    |  28 +--
 .../java/hivemall/tools/bits/BitsORUDF.java     |  28 +--
 .../java/hivemall/tools/bits/ToBitsUDF.java     |  18 ++
 .../java/hivemall/tools/bits/UnBitsUDF.java     |  18 ++
 .../hivemall/tools/compress/DeflateUDF.java     |  28 +--
 .../hivemall/tools/compress/InflateUDF.java     |  28 +--
 .../java/hivemall/tools/map/MapGetSumUDF.java   |  28 +--
 .../java/hivemall/tools/map/MapTailNUDF.java    |  28 +--
 .../main/java/hivemall/tools/map/UDAFToMap.java |  28 +--
 .../hivemall/tools/map/UDAFToOrderedMap.java    |  28 +--
 .../tools/mapred/DistributedCacheLookupUDF.java |  28 +--
 .../hivemall/tools/mapred/JobConfGetsUDF.java   |  28 +--
 .../java/hivemall/tools/mapred/JobIdUDF.java    |  28 +--
 .../java/hivemall/tools/mapred/RowIdUDF.java    |  28 +--
 .../java/hivemall/tools/mapred/TaskIdUDF.java   |  28 +--
 .../hivemall/tools/math/SigmoidGenericUDF.java  |  28 +--
 .../java/hivemall/tools/text/Base91UDF.java     |  27 +--
 .../tools/text/NormalizeUnicodeUDF.java         |  28 +--
 .../java/hivemall/tools/text/SplitWordsUDF.java |  28 +--
 .../java/hivemall/tools/text/StopwordUDF.java   |  28 +--
 .../java/hivemall/tools/text/TokenizeUDF.java   |  28 +--
 .../java/hivemall/tools/text/Unbase91UDF.java   |  27 +--
 .../main/java/hivemall/utils/HivemallUtils.java |  28 +--
 .../hivemall/utils/buffer/DynamicByteArray.java |  30 +--
 .../java/hivemall/utils/buffer/HeapBuffer.java  |  27 +--
 .../main/java/hivemall/utils/codec/Base91.java  |  27 +--
 .../hivemall/utils/codec/CompressionCodec.java  |  28 +--
 .../java/hivemall/utils/codec/DeflateCodec.java |  28 +--
 .../hivemall/utils/codec/VariableByteCodec.java |  27 +--
 .../hivemall/utils/codec/ZigZagLEB128Codec.java |  27 +--
 .../utils/collections/BoundedPriorityQueue.java |  28 +--
 .../utils/collections/DoubleArray3D.java        |  28 +--
 .../utils/collections/DoubleArrayList.java      |  27 +--
 .../utils/collections/DoubleRingBuffer.java     |  27 +--
 .../utils/collections/FixedIntArray.java        |  27 +--
 .../utils/collections/IMapIterator.java         |  28 +--
 .../hivemall/utils/collections/IndexedSet.java  |  28 +--
 .../collections/Int2FloatOpenHashTable.java     |  28 +--
 .../utils/collections/Int2IntOpenHashTable.java |  28 +--
 .../collections/Int2LongOpenHashTable.java      |  28 +--
 .../hivemall/utils/collections/IntArray.java    |  27 +--
 .../utils/collections/IntArrayList.java         |  27 +--
 .../utils/collections/IntOpenHashMap.java       |  28 +--
 .../utils/collections/IntOpenHashTable.java     |  28 +--
 .../java/hivemall/utils/collections/LRUMap.java |  28 +--
 .../hivemall/utils/collections/OpenHashMap.java |  28 +--
 .../utils/collections/OpenHashTable.java        |  28 +--
 .../hivemall/utils/collections/RingBuffer.java  |  27 +--
 .../utils/collections/SparseIntArray.java       |  27 +--
 .../utils/concurrent/ExecutorFactory.java       |  27 +--
 .../utils/concurrent/NamedThreadFactory.java    |  27 +--
 .../utils/datetime/DateTimeFormatter.java       |  28 +--
 .../java/hivemall/utils/datetime/StopWatch.java |  28 +--
 .../java/hivemall/utils/hadoop/HadoopUtils.java |  28 +--
 .../java/hivemall/utils/hadoop/HiveUtils.java   |  28 +--
 .../main/java/hivemall/utils/hadoop/Text2.java  |  27 +--
 .../main/java/hivemall/utils/hadoop/Text3.java  |  27 +--
 .../hivemall/utils/hadoop/WritableUtils.java    |  28 +--
 .../hivemall/utils/hashing/HashFunction.java    |  28 +--
 .../utils/hashing/HashFunctionFactory.java      |  28 +--
 .../hivemall/utils/hashing/MurmurHash3.java     |  28 +--
 .../utils/hashing/MurmurHash3Function.java      |  28 +--
 .../hivemall/utils/io/ASCII85InputStream.java   | 229 -------------------
 .../hivemall/utils/io/ASCII85OutputStream.java  | 150 ------------
 .../hivemall/utils/io/Base91InputStream.java    |  27 +--
 .../hivemall/utils/io/Base91OutputStream.java   |  27 +--
 .../utils/io/CompressionStreamFactory.java      |  28 +--
 .../hivemall/utils/io/DeflaterOutputStream.java |  27 +--
 .../utils/io/FastByteArrayInputStream.java      |  28 +--
 .../utils/io/FastByteArrayOutputStream.java     |  28 +--
 .../io/FastMultiByteArrayOutputStream.java      |  28 +--
 .../main/java/hivemall/utils/io/FileUtils.java  |  28 +--
 .../utils/io/FinishableOutputStream.java        |  28 +--
 .../utils/io/FinishableOutputStreamAdapter.java |  27 +--
 .../main/java/hivemall/utils/io/IOUtils.java    |  28 +--
 .../main/java/hivemall/utils/io/NIOUtils.java   |  28 +--
 .../java/hivemall/utils/io/NioFixedSegment.java |  28 +--
 .../main/java/hivemall/utils/io/NioSegment.java |  28 +--
 .../hivemall/utils/io/NioStatefullSegment.java  |  28 +--
 .../main/java/hivemall/utils/io/Segments.java   |  28 +--
 .../java/hivemall/utils/lang/ArrayUtils.java    |  28 +--
 .../main/java/hivemall/utils/lang/BitUtils.java |  28 +--
 .../hivemall/utils/lang/CommandLineUtils.java   |  28 +--
 .../main/java/hivemall/utils/lang/Copyable.java |  28 +--
 .../main/java/hivemall/utils/lang/Counter.java  |  28 +--
 .../hivemall/utils/lang/FloatAccumulator.java   |  28 +--
 .../java/hivemall/utils/lang/HalfFloat.java     |  28 +--
 .../java/hivemall/utils/lang/Identifier.java    |  28 +--
 .../java/hivemall/utils/lang/NumberUtils.java   |  28 +--
 .../java/hivemall/utils/lang/ObjectUtils.java   |  28 +--
 .../java/hivemall/utils/lang/Preconditions.java |  27 +--
 .../java/hivemall/utils/lang/Primitives.java    |  28 +--
 .../hivemall/utils/lang/PrivilegedAccessor.java |  28 +--
 .../java/hivemall/utils/lang/RandomUtils.java   |  27 +--
 .../main/java/hivemall/utils/lang/SizeOf.java   |  28 +--
 .../java/hivemall/utils/lang/StringUtils.java   |  28 +--
 .../java/hivemall/utils/lang/UnsafeUtils.java   |  27 +--
 .../utils/lang/mutable/MutableDouble.java       |  28 +--
 .../utils/lang/mutable/MutableFloat.java        |  28 +--
 .../hivemall/utils/lang/mutable/MutableInt.java |  28 +--
 .../utils/lang/mutable/MutableLong.java         |  28 +--
 .../src/main/java/hivemall/utils/lock/Lock.java |  28 +--
 .../main/java/hivemall/utils/lock/TTASLock.java |  28 +--
 .../java/hivemall/utils/math/MathUtils.java     |  28 +--
 .../java/hivemall/utils/math/MatrixUtils.java   |  27 +--
 .../main/java/hivemall/utils/math/Primes.java   |  28 +--
 .../java/hivemall/utils/math/StatsUtils.java    |  28 +--
 .../main/java/hivemall/utils/net/NetUtils.java  |  28 +--
 .../hive/ql/exec/MapredContextAccessor.java     |  28 +--
 .../hivemall/anomaly/ChangeFinder1DTest.java    |  27 +--
 .../hivemall/anomaly/ChangeFinder2DTest.java    |  27 +--
 .../classifier/PassiveAggressiveUDTFTest.java   |  28 +--
 .../hivemall/classifier/PerceptronUDTFTest.java |  28 +--
 .../hivemall/common/OnlineVarianceTest.java     |  28 +--
 .../common/RandomizedAmplifierTest.java         |  28 +--
 .../evaluation/BinaryResponsesMeasuresTest.java |  27 +--
 .../evaluation/GradedResponsesMeasuresTest.java |  27 +--
 .../test/java/hivemall/fm/ArrayModelTest.java   |  28 +--
 .../hivemall/fm/FFMPredictionModelTest.java     |  27 +--
 .../fm/FactorizationMachineUDTFTest.java        |  18 ++
 core/src/test/java/hivemall/fm/FeatureTest.java |  27 +--
 .../FieldAwareFactorizationMachineUDTFTest.java |  28 +--
 .../hivemall/fm/IntFeatureMapModelTest.java     |  28 +--
 .../hivemall/fm/StringFeatureMapModelTest.java  |  28 +--
 .../hivemall/ftvec/ExtractWeightUDFTest.java    |  27 +--
 .../java/hivemall/ftvec/FeatureUDFTest.java     |  27 +--
 .../ftvec/hashing/FeatureHashingUDFTest.java    |  28 +--
 .../ftvec/hashing/MurmurHash3UDFTest.java       |  28 +--
 .../hivemall/ftvec/hashing/Sha1UDFTest.java     |  28 +--
 .../pairing/PolynomialFeaturesUDFTest.java      |  28 +--
 .../ftvec/pairing/PoweredFeaturesUDFTest.java   |  28 +--
 .../ftvec/scaling/L2NormalizationUDFTest.java   |  28 +--
 .../ftvec/trans/TestBinarizeLabelUDTF.java      |  27 +--
 .../ftvec/trans/VectorizeFeaturesUDFTest.java   |  28 +--
 .../knn/distance/EuclidDistanceUDFTest.java     |  28 +--
 .../java/hivemall/knn/lsh/MinHashUDFTest.java   |  28 +--
 .../hivemall/knn/lsh/bBitMinHashUDFTest.java    |  28 +--
 .../knn/similarity/CosineSimilarityUDFTest.java |  28 +--
 .../mf/BPRMatrixFactorizationUDTFTest.java      |  28 +--
 .../mf/MatrixFactorizationAdaGradUDTFTest.java  |  28 +--
 .../mf/MatrixFactorizationSGDUDTFTest.java      |  28 +--
 .../mix/client/MixRequestRouterTest.java        |  28 +--
 .../java/hivemall/model/FeatureValueTest.java   |  28 +--
 .../model/SpaceEfficientDenseModelTest.java     |  28 +--
 .../hivemall/regression/AdaGradUDTFTest.java    |  28 +--
 .../smile/classification/DecisionTreeTest.java  |  28 +--
 .../RandomForestClassifierUDTFTest.java         |  28 +--
 .../smile/regression/RegressionTreeTest.java    |  28 +--
 .../smile/tools/TreePredictUDFTest.java         |  28 +--
 .../hivemall/smile/vm/StackMachineTest.java     |  28 +--
 .../tools/array/ToStringArrayUDFTest.java       |  28 +--
 .../java/hivemall/utils/ArrayUtilsTest.java     |  28 +--
 .../test/java/hivemall/utils/BitUtilsTest.java  |  28 +--
 .../hivemall/utils/buffer/HeapBufferTest.java   |  27 +--
 .../java/hivemall/utils/codec/Base91Test.java   |  27 +--
 .../hivemall/utils/codec/DeflateCodecTest.java  |  28 +--
 .../utils/codec/ZigZagLEB128CodecTest.java      |  18 ++
 .../utils/collections/DoubleArray3DTest.java    |  27 +--
 .../utils/collections/DoubleRingBufferTest.java |  27 +--
 .../collections/Int2FloatOpenHashMapTest.java   |  28 +--
 .../collections/Int2LongOpenHashMapTest.java    |  28 +--
 .../utils/collections/IntOpenHashMapTest.java   |  28 +--
 .../utils/collections/IntOpenHashTableTest.java |  28 +--
 .../utils/collections/OpenHashMapTest.java      |  28 +--
 .../utils/collections/OpenHashTableTest.java    |  28 +--
 .../utils/collections/SparseIntArrayTest.java   |  18 ++
 .../hivemall/utils/hadoop/HadoopUtilsTest.java  |  28 +--
 .../java/hivemall/utils/hadoop/Text2Test.java   |  28 +--
 .../hivemall/utils/hashing/MurmurHash3Test.java |  28 +--
 .../utils/io/Base91OutputStreamTest.java        |  27 +--
 .../java/hivemall/utils/lang/HalfFloatTest.java |  28 +--
 .../hivemall/utils/lang/NumberUtilsTest.java    |  28 +--
 .../hivemall/utils/math/MatrixUtilsTest.java    |  27 +--
 docs/gitbook/README.md                          |  19 ++
 docs/gitbook/SUMMARY.md                         |  19 ++
 docs/gitbook/anomaly/lof.md                     |  19 ++
 docs/gitbook/binaryclass/a9a.md                 |  19 ++
 docs/gitbook/binaryclass/a9a_dataset.md         |  19 ++
 docs/gitbook/binaryclass/a9a_lr.md              |  19 ++
 docs/gitbook/binaryclass/a9a_minibatch.md       |  19 ++
 docs/gitbook/binaryclass/kdd2010a.md            |  19 ++
 docs/gitbook/binaryclass/kdd2010a_dataset.md    |  19 ++
 docs/gitbook/binaryclass/kdd2010a_scw.md        |  19 ++
 docs/gitbook/binaryclass/kdd2010b.md            |  19 ++
 docs/gitbook/binaryclass/kdd2010b_arow.md       |  19 ++
 docs/gitbook/binaryclass/kdd2010b_dataset.md    |  19 ++
 docs/gitbook/binaryclass/news20.md              |  19 ++
 docs/gitbook/binaryclass/news20_adagrad.md      |  19 ++
 docs/gitbook/binaryclass/news20_dataset.md      |  19 ++
 docs/gitbook/binaryclass/news20_pa.md           |  19 ++
 docs/gitbook/binaryclass/news20_scw.md          |  19 ++
 docs/gitbook/binaryclass/webspam.md             |  19 ++
 docs/gitbook/binaryclass/webspam_dataset.md     |  19 ++
 docs/gitbook/binaryclass/webspam_scw.md         |  19 ++
 docs/gitbook/eval/datagen.md                    |  19 ++
 docs/gitbook/eval/lr_datagen.md                 |  19 ++
 docs/gitbook/eval/stat_eval.md                  |  19 ++
 docs/gitbook/ft_engineering/ft_trans.md         |  19 ++
 docs/gitbook/ft_engineering/hashing.md          |  19 ++
 docs/gitbook/ft_engineering/quantify.md         |  19 ++
 docs/gitbook/ft_engineering/scaling.md          |  19 ++
 docs/gitbook/ft_engineering/tfidf.md            |  19 ++
 docs/gitbook/ft_engineering/vectorizer.md       |  19 ++
 docs/gitbook/getting_started/README.md          |  19 ++
 docs/gitbook/getting_started/input-format.md    |  19 ++
 docs/gitbook/getting_started/installation.md    |  19 ++
 .../getting_started/permanent-functions.md      |  19 ++
 docs/gitbook/misc/generic_funcs.md              |  19 ++
 docs/gitbook/misc/tokenizer.md                  |  19 ++
 docs/gitbook/misc/topk.md                       |  19 ++
 docs/gitbook/multiclass/iris.md                 |  19 ++
 docs/gitbook/multiclass/iris_dataset.md         |  19 ++
 docs/gitbook/multiclass/iris_randomforest.md    |  19 ++
 docs/gitbook/multiclass/iris_scw.md             |  19 ++
 docs/gitbook/multiclass/news20.md               |  19 ++
 docs/gitbook/multiclass/news20_dataset.md       |  19 ++
 docs/gitbook/multiclass/news20_ensemble.md      |  19 ++
 .../multiclass/news20_one-vs-the-rest.md        |  19 ++
 .../news20_one-vs-the-rest_dataset.md           |  19 ++
 docs/gitbook/multiclass/news20_pa.md            |  19 ++
 docs/gitbook/multiclass/news20_scw.md           |  19 ++
 docs/gitbook/recommend/cf.md                    |  19 ++
 docs/gitbook/recommend/item_based_cf.md         |  19 ++
 docs/gitbook/recommend/movielens.md             |  19 ++
 docs/gitbook/recommend/movielens_cv.md          |  19 ++
 docs/gitbook/recommend/movielens_dataset.md     |  19 ++
 docs/gitbook/recommend/movielens_fm.md          |  19 ++
 docs/gitbook/recommend/movielens_mf.md          |  19 ++
 docs/gitbook/recommend/news20.md                |  19 ++
 docs/gitbook/recommend/news20_bbit_minhash.md   |  19 ++
 docs/gitbook/recommend/news20_jaccard.md        |  19 ++
 docs/gitbook/recommend/news20_knn.md            |  19 ++
 docs/gitbook/regression/e2006.md                |  19 ++
 docs/gitbook/regression/e2006_arow.md           |  19 ++
 docs/gitbook/regression/e2006_dataset.md        |  19 ++
 docs/gitbook/regression/kddcup12tr2.md          |  19 ++
 docs/gitbook/regression/kddcup12tr2_adagrad.md  |  19 ++
 docs/gitbook/regression/kddcup12tr2_dataset.md  |  19 ++
 docs/gitbook/regression/kddcup12tr2_lr.md       |  19 ++
 .../regression/kddcup12tr2_lr_amplify.md        |  19 ++
 docs/gitbook/tips/README.md                     |  19 ++
 docs/gitbook/tips/addbias.md                    |  19 ++
 docs/gitbook/tips/emr.md                        |  19 ++
 docs/gitbook/tips/ensemble_learning.md          |  19 ++
 docs/gitbook/tips/general_tips.md               |  19 ++
 docs/gitbook/tips/hadoop_tuning.md              |  19 ++
 docs/gitbook/tips/mixserver.md                  |  19 ++
 docs/gitbook/tips/rand_amplify.md               |  19 ++
 docs/gitbook/tips/rowid.md                      |  19 ++
 docs/gitbook/tips/rt_prediction.md              |  19 ++
 docs/gitbook/troubleshooting/README.md          |  19 ++
 docs/gitbook/troubleshooting/asterisk.md        |  19 ++
 .../troubleshooting/mapjoin_classcastex.md      |  19 ++
 .../troubleshooting/mapjoin_task_error.md       |  19 ++
 docs/gitbook/troubleshooting/num_mappers.md     |  19 ++
 docs/gitbook/troubleshooting/oom.md             |  19 ++
 mixserv/pom.xml                                 |  42 ++--
 .../hivemall/mix/metrics/MetricsRegistry.java   |  28 +--
 .../hivemall/mix/metrics/MixServerMetrics.java  |  28 +--
 .../mix/metrics/MixServerMetricsMBean.java      |  28 +--
 .../hivemall/mix/metrics/ThroughputCounter.java |  28 +--
 .../java/hivemall/mix/server/MixServer.java     |  28 +--
 .../hivemall/mix/server/MixServerHandler.java   |  28 +--
 .../mix/server/MixServerInitializer.java        |  28 +--
 .../hivemall/mix/store/PartialArgminKLD.java    |  28 +--
 .../java/hivemall/mix/store/PartialAverage.java |  28 +--
 .../java/hivemall/mix/store/PartialResult.java  |  28 +--
 .../java/hivemall/mix/store/SessionObject.java  |  28 +--
 .../java/hivemall/mix/store/SessionStore.java   |  28 +--
 .../mix/server/MixServerHandlerTest.java        |  28 +--
 .../java/hivemall/mix/server/MixServerTest.java |  28 +--
 .../hivemall/mix/server/PartialResultTest.java  |  28 +--
 .../java/hivemall/test/HivemallTestBase.java    |  28 +--
 nlp/pom.xml                                     |  42 ++--
 .../hivemall/nlp/tokenizer/KuromojiUDF.java     |  28 +--
 .../hivemall/nlp/tokenizer/KuromojiUDFTest.java |  28 +--
 pom.xml                                         |  68 +++++-
 resources/header-definition.xml                 |  11 +
 resources/license-header.txt                    |  28 +--
 .../org/apache/spark/sql/hive/HiveShim.scala    |  27 +--
 spark/spark-1.6/pom.xml                         |  19 ++
 .../hivemall/tools/RegressionDatagen.scala      |  27 +--
 .../apache/spark/sql/hive/GroupedDataEx.scala   |  27 +--
 .../org/apache/spark/sql/hive/HivemallOps.scala |  27 +--
 .../apache/spark/sql/hive/HivemallUtils.scala   |  27 +--
 .../hivemall/mix/server/MixServerSuite.scala    |  27 +--
 .../hivemall/tools/RegressionDatagenSuite.scala |  27 +--
 .../scala/org/apache/spark/SparkFunSuite.scala  |  27 +--
 .../ml/feature/HivemallLabeledPointSuite.scala  |  27 +--
 .../scala/org/apache/spark/sql/QueryTest.scala  |  27 +--
 .../spark/sql/catalyst/plans/PlanTest.scala     |  27 +--
 .../apache/spark/sql/hive/HiveUdfSuite.scala    |  27 +--
 .../spark/sql/hive/HivemallOpsSuite.scala       |  27 +--
 .../spark/sql/hive/ModelMixingSuite.scala       |  27 +--
 .../spark/streaming/HivemallOpsSuite.scala      |  27 +--
 .../apache/spark/test/HivemallQueryTest.scala   |  27 +--
 .../scala/org/apache/spark/test/TestUtils.scala |  27 +--
 .../org/apache/spark/sql/hive/HiveShim.scala    |  27 +--
 spark/spark-2.0/pom.xml                         |  19 ++
 .../java/hivemall/xgboost/XGBoostOptions.scala  |  27 +--
 .../XGBoostBinaryClassifierUDTFWrapper.java     |  27 +--
 .../XGBoostMulticlassClassifierUDTFWrapper.java |  27 +--
 .../main/java/hivemall/xgboost/package.scala    |  29 ++-
 .../XGBoostRegressionUDTFWrapper.java           |  27 +--
 .../hivemall/tools/RegressionDatagen.scala      |  27 +--
 .../sql/catalyst/expressions/EachTopK.scala     |  27 +--
 .../spark/sql/hive/HivemallGroupedDataset.scala |  27 +--
 .../org/apache/spark/sql/hive/HivemallOps.scala |  27 +--
 .../apache/spark/sql/hive/HivemallUtils.scala   |  27 +--
 .../sql/hive/source/XGBoostFileFormat.scala     |  27 +--
 .../hivemall/mix/server/MixServerSuite.scala    |  27 +--
 .../hivemall/tools/RegressionDatagenSuite.scala |  27 +--
 .../scala/org/apache/spark/SparkFunSuite.scala  |  27 +--
 .../ml/feature/HivemallLabeledPointSuite.scala  |  27 +--
 .../scala/org/apache/spark/sql/QueryTest.scala  |  27 +--
 .../spark/sql/catalyst/plans/PlanTest.scala     |  27 +--
 .../apache/spark/sql/hive/HiveUdfSuite.scala    |  27 +--
 .../spark/sql/hive/HivemallOpsSuite.scala       |  27 +--
 .../spark/sql/hive/ModelMixingSuite.scala       |  27 +--
 .../apache/spark/sql/hive/XGBoostSuite.scala    |  27 +--
 .../sql/hive/benchmark/MiscBenchmark.scala      |  27 +--
 .../spark/sql/hive/test/TestHiveSingleton.scala |  27 +--
 .../streaming/HivemallFeatureOpsSuite.scala     |  27 +--
 .../spark/test/HivemallFeatureQueryTest.scala   |  27 +--
 .../scala/org/apache/spark/test/TestUtils.scala |  27 +--
 .../org/apache/spark/test/VectorQueryTest.scala |  27 +--
 spark/spark-common/pom.xml                      |  19 ++
 spark/spark-common/scalastyle-config.xml        |  60 ++---
 ...isticRegressionDataGeneratorUDTFWrapper.java |  27 +--
 .../java/hivemall/ftvec/AddBiasUDFWrapper.java  |  27 +--
 .../ftvec/AddFeatureIndexUDFWrapper.java        |  27 +--
 .../ftvec/ExtractFeatureUDFWrapper.java         |  27 +--
 .../hivemall/ftvec/ExtractWeightUDFWrapper.java |  27 +--
 .../hivemall/ftvec/SortByFeatureUDFWrapper.java |  27 +--
 .../scaling/L2NormalizationUDFWrapper.java      |  27 +--
 .../hivemall/knn/lsh/MinHashesUDFWrapper.java   |  27 +--
 .../hivemall/tools/mapred/RowIdUDFWrapper.java  |  27 +--
 .../main/scala/hivemall/HivemallException.scala |  27 +--
 .../spark/ml/feature/HivemallLabeledPoint.scala |  27 +--
 .../spark/streaming/HivemallStreamingOps.scala  |  27 +--
 src/site/markdown/404.md                        |  19 ++
 src/site/markdown/contributing.md               |  19 ++
 src/site/markdown/download.md                   |  19 ++
 src/site/markdown/faq.md                        |  19 ++
 src/site/markdown/overview.md                   |  33 ++-
 src/site/markdown/poweredby.md                  |  19 ++
 src/site/markdown/repository.md                 |  33 ++-
 src/site/markdown/userguide.md                  |  19 ++
 xgboost/pom.xml                                 |  39 ++--
 .../java/hivemall/xgboost/NativeLibLoader.java  |  28 +--
 .../hivemall/xgboost/XGBoostPredictUDTF.java    |  28 +--
 .../main/java/hivemall/xgboost/XGBoostUDTF.java |  28 +--
 .../java/hivemall/xgboost/XGBoostUtils.java     |  28 +--
 .../XGBoostBinaryClassifierUDTF.java            |  28 +--
 .../XGBoostMulticlassClassifierUDTF.java        |  28 +--
 .../regression/XGBoostRegressionUDTF.java       |  28 +--
 .../tools/XGBoostMulticlassPredictUDTF.java     |  28 +--
 .../xgboost/tools/XGBoostPredictUDTF.java       |  28 +--
 563 files changed, 8484 insertions(+), 6681 deletions(-)
----------------------------------------------------------------------



[23/50] [abbrv] incubator-hivemall git commit: HIVEMALL-14: Build instruction including xgboost library (#378)

Posted by my...@apache.org.
HIVEMALL-14: Build instruction including xgboost library (#378)



Project: http://git-wip-us.apache.org/repos/asf/incubator-hivemall/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-hivemall/commit/9a6d1c03
Tree: http://git-wip-us.apache.org/repos/asf/incubator-hivemall/tree/9a6d1c03
Diff: http://git-wip-us.apache.org/repos/asf/incubator-hivemall/diff/9a6d1c03

Branch: refs/heads/master
Commit: 9a6d1c03d011dc56d36c3b101fe64015028e5c82
Parents: 5ebdc06
Author: Kai Sasaki <le...@me.com>
Authored: Sun Oct 30 19:24:59 2016 +0900
Committer: Makoto YUI <yu...@gmail.com>
Committed: Sun Oct 30 19:24:59 2016 +0900

----------------------------------------------------------------------
 src/site/markdown/contributing.md | 18 ++++++++++++++++++
 1 file changed, 18 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/9a6d1c03/src/site/markdown/contributing.md
----------------------------------------------------------------------
diff --git a/src/site/markdown/contributing.md b/src/site/markdown/contributing.md
index 5668971..17f7020 100644
--- a/src/site/markdown/contributing.md
+++ b/src/site/markdown/contributing.md
@@ -1,3 +1,21 @@
+# How to build
+
+#### Prerequisites
+
+* Maven 3.x
+* JDK 1.7 or later
+
+```
+$ cd hivemall
+
+# This is a workaround for resolving xgboost dependencies.
+$ mvn validate -Pxgboost
+ 
+$ mvn clean package
+```
+
+# Contribution guideline
+
 to appear ...
 
 [Here](http://www.apache.org/foundation/getinvolved.html) is a general guide for contributing to Apache Project.


[09/50] [abbrv] incubator-hivemall git commit: Added learn more button

Posted by my...@apache.org.
Added learn more button

Project: http://git-wip-us.apache.org/repos/asf/incubator-hivemall/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-hivemall/commit/f443c065
Tree: http://git-wip-us.apache.org/repos/asf/incubator-hivemall/tree/f443c065
Diff: http://git-wip-us.apache.org/repos/asf/incubator-hivemall/diff/f443c065

Branch: refs/heads/master
Commit: f443c065ad6fc71550526dacd1b47a1f78b11f94
Parents: 77d783a
Author: myui <yu...@gmail.com>
Authored: Mon Oct 24 11:34:56 2016 +0900
Committer: myui <yu...@gmail.com>
Committed: Mon Oct 24 11:34:56 2016 +0900

----------------------------------------------------------------------
 src/site/xdoc/index.xml.vm | 1 +
 1 file changed, 1 insertion(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/f443c065/src/site/xdoc/index.xml.vm
----------------------------------------------------------------------
diff --git a/src/site/xdoc/index.xml.vm b/src/site/xdoc/index.xml.vm
index 2dedfca..816d8f8 100644
--- a/src/site/xdoc/index.xml.vm
+++ b/src/site/xdoc/index.xml.vm
@@ -28,6 +28,7 @@
     <body>
         <div class="alert alert-info" role="alert">
             <strong>Info:</strong> We are now in the process of migrating the project repository from <a href="https://github.com/myui/hivemall">Github</a> to <a href="https://github.com/apache/incubator-hivemall">Apache Incubator</a>.
+            <a href="/userguide/index.html"><button class="btn btn-primary" type="button">Learn more</button></a>
         </div>
         <div id="carousel-main" class="row">
             <div id="screenshots-carousel" class="carousel slide span10">


[30/50] [abbrv] incubator-hivemall git commit: Updated license headers of src/site/markdown/*.md

Posted by my...@apache.org.
Updated license headers of src/site/markdown/*.md

Project: http://git-wip-us.apache.org/repos/asf/incubator-hivemall/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-hivemall/commit/63aa3b91
Tree: http://git-wip-us.apache.org/repos/asf/incubator-hivemall/tree/63aa3b91
Diff: http://git-wip-us.apache.org/repos/asf/incubator-hivemall/diff/63aa3b91

Branch: refs/heads/master
Commit: 63aa3b91f0aa5d55fe7346ea5d9f7e080d845f34
Parents: 67bdcb8
Author: myui <yu...@gmail.com>
Authored: Mon Nov 7 15:54:12 2016 +0900
Committer: myui <yu...@gmail.com>
Committed: Mon Nov 7 15:54:12 2016 +0900

----------------------------------------------------------------------
 pom.xml                           |  3 ++-
 src/site/markdown/404.md          | 19 +++++++++++++++++++
 src/site/markdown/contributing.md | 19 +++++++++++++++++++
 src/site/markdown/download.md     | 19 +++++++++++++++++++
 src/site/markdown/faq.md          | 19 +++++++++++++++++++
 src/site/markdown/overview.md     | 33 ++++++++++++++++-----------------
 src/site/markdown/poweredby.md    | 19 +++++++++++++++++++
 src/site/markdown/repository.md   | 33 ++++++++++++++++-----------------
 src/site/markdown/userguide.md    | 19 +++++++++++++++++++
 9 files changed, 148 insertions(+), 35 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/63aa3b91/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 21a9a57..2653cb7 100644
--- a/pom.xml
+++ b/pom.xml
@@ -415,8 +415,9 @@
 							<include>src/main/**/*.scala</include>
 							<include>src/test/**/*.scala</include>
 							<include>bin/*.sh</include>
-							<include>docs/gitbook/**/*.md</include>
 							<include>**/pom.xml</include>
+							<include>docs/gitbook/**/*.md</include>
+							<include>src/site/markdown/**/*.md</include>
 						</includes>
 						<useDefaultExcludes>false</useDefaultExcludes>
 						<excludes>

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/63aa3b91/src/site/markdown/404.md
----------------------------------------------------------------------
diff --git a/src/site/markdown/404.md b/src/site/markdown/404.md
index 45ec269..68d09f4 100644
--- a/src/site/markdown/404.md
+++ b/src/site/markdown/404.md
@@ -1 +1,20 @@
+<!--
+  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.
+-->
+
 Currently under construction.
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/63aa3b91/src/site/markdown/contributing.md
----------------------------------------------------------------------
diff --git a/src/site/markdown/contributing.md b/src/site/markdown/contributing.md
index 5668971..999380a 100644
--- a/src/site/markdown/contributing.md
+++ b/src/site/markdown/contributing.md
@@ -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.
+-->
+
 to appear ...
 
 [Here](http://www.apache.org/foundation/getinvolved.html) is a general guide for contributing to Apache Project.

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/63aa3b91/src/site/markdown/download.md
----------------------------------------------------------------------
diff --git a/src/site/markdown/download.md b/src/site/markdown/download.md
index 9cbe300..115cd60 100644
--- a/src/site/markdown/download.md
+++ b/src/site/markdown/download.md
@@ -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.
+-->
+
 No release in Apache Incubator yet.
 
 Past releases of Hivemall can be found in [this page](https://github.com/myui/hivemall/releases).

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/63aa3b91/src/site/markdown/faq.md
----------------------------------------------------------------------
diff --git a/src/site/markdown/faq.md b/src/site/markdown/faq.md
index f04a65b..5a8b1fb 100644
--- a/src/site/markdown/faq.md
+++ b/src/site/markdown/faq.md
@@ -1 +1,20 @@
+<!--
+  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.
+-->
+
 to appear ...
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/63aa3b91/src/site/markdown/overview.md
----------------------------------------------------------------------
diff --git a/src/site/markdown/overview.md b/src/site/markdown/overview.md
index 5e26e60..57c8d73 100644
--- a/src/site/markdown/overview.md
+++ b/src/site/markdown/overview.md
@@ -1,21 +1,20 @@
 <!--
-
-	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.
-
+  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.
 -->
 
 # Apache Hivemall Overview

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/63aa3b91/src/site/markdown/poweredby.md
----------------------------------------------------------------------
diff --git a/src/site/markdown/poweredby.md b/src/site/markdown/poweredby.md
index f04a65b..5a8b1fb 100644
--- a/src/site/markdown/poweredby.md
+++ b/src/site/markdown/poweredby.md
@@ -1 +1,20 @@
+<!--
+  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.
+-->
+
 to appear ...
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/63aa3b91/src/site/markdown/repository.md
----------------------------------------------------------------------
diff --git a/src/site/markdown/repository.md b/src/site/markdown/repository.md
index 0f2d716..2beecd9 100644
--- a/src/site/markdown/repository.md
+++ b/src/site/markdown/repository.md
@@ -1,21 +1,20 @@
 <!--
-
-	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.
-
+  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.
 -->
 
 # Source

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/63aa3b91/src/site/markdown/userguide.md
----------------------------------------------------------------------
diff --git a/src/site/markdown/userguide.md b/src/site/markdown/userguide.md
index cd1a4dc..3cff047 100644
--- a/src/site/markdown/userguide.md
+++ b/src/site/markdown/userguide.md
@@ -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.
+-->
+
 Yet under construction.
 
 Please refer [this wiki page](https://github.com/myui/hivemall/wiki) for the time being.


[06/50] [abbrv] incubator-hivemall git commit: Removed a PDF link and the fixed site building script

Posted by my...@apache.org.
Removed a PDF link and the fixed site building script


Project: http://git-wip-us.apache.org/repos/asf/incubator-hivemall/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-hivemall/commit/eee79301
Tree: http://git-wip-us.apache.org/repos/asf/incubator-hivemall/tree/eee79301
Diff: http://git-wip-us.apache.org/repos/asf/incubator-hivemall/diff/eee79301

Branch: refs/heads/master
Commit: eee7930160d4b93be278701b998f3b0fd9b21e2a
Parents: 11d5f01 ef13a62
Author: myui <yu...@gmail.com>
Authored: Sun Oct 23 04:33:14 2016 +0900
Committer: myui <yu...@gmail.com>
Committed: Sun Oct 23 04:33:14 2016 +0900

----------------------------------------------------------------------
 bin/build_site.sh       | 2 ++
 docs/gitbook/.gitignore | 1 +
 docs/gitbook/book.json  | 1 -
 3 files changed, 3 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/eee79301/bin/build_site.sh
----------------------------------------------------------------------
diff --cc bin/build_site.sh
index 5eb97f7,5eb97f7..bad6548
--- a/bin/build_site.sh
+++ b/bin/build_site.sh
@@@ -14,6 -14,6 +14,8 @@@ f
  cd $HIVEMALL_HOME
  mvn clean site
  
++cp -R docs/gitbook/_book target/site/userguide
++
  #
  # Run HTTP server on localhost
  #


[40/50] [abbrv] incubator-hivemall git commit: Updated site contents

Posted by my...@apache.org.
Updated site contents

Project: http://git-wip-us.apache.org/repos/asf/incubator-hivemall/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-hivemall/commit/52bc44fa
Tree: http://git-wip-us.apache.org/repos/asf/incubator-hivemall/tree/52bc44fa
Diff: http://git-wip-us.apache.org/repos/asf/incubator-hivemall/diff/52bc44fa

Branch: refs/heads/master
Commit: 52bc44fa258b1595bf2e8e9e9e71eace783a6e5b
Parents: 32a657d
Author: myui <yu...@gmail.com>
Authored: Mon Nov 14 19:47:47 2016 +0900
Committer: myui <yu...@gmail.com>
Committed: Mon Nov 14 19:47:47 2016 +0900

----------------------------------------------------------------------
 bin/build_site.sh                               |  26 ++++++++++++++++---
 docs/gitbook/FOOTER.md                          |   3 +++
 docs/gitbook/README.md                          |  10 ++-----
 docs/gitbook/book.json                          |   3 ++-
 .../regression/kddcup12tr2_lr_amplify.md        |  12 ++++-----
 docs/gitbook/resources/images/amplify.png       | Bin 0 -> 94408 bytes
 .../resources/images/amplify_elapsed.png        | Bin 0 -> 19556 bytes
 docs/gitbook/resources/images/emr-bootstrap.png | Bin 0 -> 31830 bytes
 docs/gitbook/resources/images/emr-wizard.png    | Bin 0 -> 35576 bytes
 docs/gitbook/resources/images/randamplify.png   | Bin 0 -> 49567 bytes
 .../resources/images/randamplify_elapsed.png    | Bin 0 -> 19047 bytes
 docs/gitbook/tips/emr.md                        |   6 +++--
 docs/gitbook/tips/rand_amplify.md               |  11 ++++----
 13 files changed, 45 insertions(+), 26 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/52bc44fa/bin/build_site.sh
----------------------------------------------------------------------
diff --git a/bin/build_site.sh b/bin/build_site.sh
index 04496b4..de3aa08 100755
--- a/bin/build_site.sh
+++ b/bin/build_site.sh
@@ -30,16 +30,36 @@ if [ "$HIVEMALL_HOME" == "" ]; then
 fi
 
 cd $HIVEMALL_HOME
+HIVEMALL_HOME=`pwd`
+
+##
+# Run maven-site
+##
+
 mvn clean site
 
+##
+# building gitbook userguide
+##
+
+if ! [ -x "$(command -v gitbook)" ]; then
+  echo "gitbook is not installed .." >&2
+  echo "Run 'npm install gitbook-cli -g' to install gitbook" >&2
+  exit 1
+fi
+
+cd ${HIVEMALL_HOME}/docs/gitbook
+gitbook install && gitbook build
+cd $HIVEMALL_HOME
+
 cp -R docs/gitbook/_book target/site/userguide
 
-#
+##
 # Run HTTP server on localhost
-#
+##
 
 # ruby
-cd $HIVEMALL_HOME/target/site
+cd ${HIVEMALL_HOME}/target/site
 ruby -rwebrick -e 'WEBrick::HTTPServer.new(:DocumentRoot => "./", :Port => 8000).start'
 
 # python3

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/52bc44fa/docs/gitbook/FOOTER.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/FOOTER.md b/docs/gitbook/FOOTER.md
new file mode 100644
index 0000000..588afbb
--- /dev/null
+++ b/docs/gitbook/FOOTER.md
@@ -0,0 +1,3 @@
+<sub><font color="gray">
+Apache Hivemall is an effort undergoing incubation at The Apache Software Foundation (ASF), sponsored by the Apache Incubator.
+</font></sub>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/52bc44fa/docs/gitbook/README.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/README.md b/docs/gitbook/README.md
index 7b61570..164ef74 100644
--- a/docs/gitbook/README.md
+++ b/docs/gitbook/README.md
@@ -23,7 +23,7 @@
 Apache Hivemall is a collection of machine learning algorithms and versatile data analytics functions. It provides a number of ease of use machine learning functionalities through the <a href="https://cwiki.apache.org/confluence/display/Hive/LanguageManual+UDF">Apache Hive UDF/UDAF/UDTF interface</a>.
 </div>
 
-<div style="text-align:center"><img src="resources/images/hivemall-logo-color-small.png"/></div>
+<div style="text-align:center"><img src="./resources/images/hivemall-logo-color-small.png"/></div>
 
 Apache Hivemall offers a variety of functionalities: <strong>regression, classification, recommendation, anomaly detection, k-nearest neighbor, and feature engineering</strong>. It also supports state-of-the-art machine learning algorithms such as Soft Confidence Weighted, Adaptive Regularization of Weight Vectors, Factorization Machines, and AdaDelta. 
 
@@ -32,10 +32,4 @@ Apache Hivemall offers a variety of functionalities: <strong>regression, classif
 Apache Hivemall is mainly designed to run on [Apache Hive](https://hive.apache.org/) but it also supports [Apache Pig](https://pig.apache.org/) and [Apache Spark](http://spark.apache.org/) for the runtime.
 Thus, it can be considered as a cross platform library for machine learning; prediction models built by a batch query of Apache Hive can be used on Apache Spark/Pig, and conversely, prediction models build by Apache Spark can be used from Apache Hive/Pig.
 
-<div style="text-align:center"><img src="resources/images/techstack.png" width="80%" height="80%"/></div>
-
----
-
-<font color="gray">
-<sub>Apache Hivemall is an effort undergoing incubation at The Apache Software Foundation (ASF), sponsored by the <a href="http://incubator.apache.org/">Apache Incubator</a>.</sub>
-</font>
\ No newline at end of file
+<div style="text-align:center"><img src="./resources/images/techstack.png" width="80%" height="80%"/></div>

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/52bc44fa/docs/gitbook/book.json
----------------------------------------------------------------------
diff --git a/docs/gitbook/book.json b/docs/gitbook/book.json
index 2f70ed9..b622a7b 100644
--- a/docs/gitbook/book.json
+++ b/docs/gitbook/book.json
@@ -17,7 +17,8 @@
         "multipart",
         "codeblock-filename",
         "katex",
-        "emphasize"
+        "emphasize",
+        "localized-footer"
     ],
     "pluginsConfig": {
         "theme-default": {

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/52bc44fa/docs/gitbook/regression/kddcup12tr2_lr_amplify.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/regression/kddcup12tr2_lr_amplify.md b/docs/gitbook/regression/kddcup12tr2_lr_amplify.md
index 55b8caf..e402ce4 100644
--- a/docs/gitbook/regression/kddcup12tr2_lr_amplify.md
+++ b/docs/gitbook/regression/kddcup12tr2_lr_amplify.md
@@ -70,8 +70,8 @@ group by feature;
 ```
 
 The above query is executed by 2 MapReduce jobs as shown below:
-![amplifier](https://dl.dropboxusercontent.com/u/13123103/hivemall/amplify.png)
-[Here](https://dl.dropboxusercontent.com/u/13123103/hivemall/amplify_plan.txt) is the actual plan generated by the Hive.
+
+<img src="../resources/images/amplify.png" alt="amplifier"/>
 
 Using *trainning_x3*  instead of the plain training table results in higher and better AUC (0.746214) in [this](https://github.com/myui/hivemall/wiki/KDDCup-2012-track-2-CTR-prediction-(regression\)) example.
 
@@ -80,7 +80,7 @@ When the training table is so large that involves 100 Map tasks, the merge opera
 
 Note that the actual bottleneck is not M/R iterations but shuffling training instance. Iteration without shuffling (as in [the Spark example](http://spark.incubator.apache.org/examples.html)) causes very slow convergence and results in requiring more iterations. Shuffling cannot be avoided even in iterative MapReduce variants.
 
-![amplify elapsed](https://dl.dropboxusercontent.com/u/13123103/hivemall/amplify_elapsed.png)
+<img src="../resources/images/amplify_elapsed.png" alt="amplify elapsed"/>
 
 ---
 # Amplify and shuffle training examples in each Map task
@@ -101,12 +101,12 @@ from
 ```
 
 The training query is executed as follows:
-![Random amplify](https://dl.dropboxusercontent.com/u/13123103/hivemall/randamplify.png)  
-[Here](https://dl.dropboxusercontent.com/u/13123103/hivemall/randamplify_plan.txt) is the actual query plan.
+
+<img src="../resources/images/randamplify.png" alt="Random amplify"/>
 
 The map-local multiplication and shuffling has no bottleneck in the merge phase and the query is efficiently executed within a single MapReduce job.
 
-![rand_amplify elapsed ](https://dl.dropboxusercontent.com/u/13123103/hivemall/randamplify_elapsed.png)
+<img src="../resources/images/randamplify_elapsed.png" alt="rand_amplify elapsed"/>
 
 Using *rand_amplify* results in a better AUC (0.743392) in [this](https://github.com/myui/hivemall/wiki/KDDCup-2012-track-2-CTR-prediction-(regression\)) example.
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/52bc44fa/docs/gitbook/resources/images/amplify.png
----------------------------------------------------------------------
diff --git a/docs/gitbook/resources/images/amplify.png b/docs/gitbook/resources/images/amplify.png
new file mode 100644
index 0000000..f537e98
Binary files /dev/null and b/docs/gitbook/resources/images/amplify.png differ

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/52bc44fa/docs/gitbook/resources/images/amplify_elapsed.png
----------------------------------------------------------------------
diff --git a/docs/gitbook/resources/images/amplify_elapsed.png b/docs/gitbook/resources/images/amplify_elapsed.png
new file mode 100644
index 0000000..595dd60
Binary files /dev/null and b/docs/gitbook/resources/images/amplify_elapsed.png differ

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/52bc44fa/docs/gitbook/resources/images/emr-bootstrap.png
----------------------------------------------------------------------
diff --git a/docs/gitbook/resources/images/emr-bootstrap.png b/docs/gitbook/resources/images/emr-bootstrap.png
new file mode 100644
index 0000000..fea2ee2
Binary files /dev/null and b/docs/gitbook/resources/images/emr-bootstrap.png differ

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/52bc44fa/docs/gitbook/resources/images/emr-wizard.png
----------------------------------------------------------------------
diff --git a/docs/gitbook/resources/images/emr-wizard.png b/docs/gitbook/resources/images/emr-wizard.png
new file mode 100644
index 0000000..725cc9e
Binary files /dev/null and b/docs/gitbook/resources/images/emr-wizard.png differ

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/52bc44fa/docs/gitbook/resources/images/randamplify.png
----------------------------------------------------------------------
diff --git a/docs/gitbook/resources/images/randamplify.png b/docs/gitbook/resources/images/randamplify.png
new file mode 100644
index 0000000..432f775
Binary files /dev/null and b/docs/gitbook/resources/images/randamplify.png differ

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/52bc44fa/docs/gitbook/resources/images/randamplify_elapsed.png
----------------------------------------------------------------------
diff --git a/docs/gitbook/resources/images/randamplify_elapsed.png b/docs/gitbook/resources/images/randamplify_elapsed.png
new file mode 100644
index 0000000..7d5be32
Binary files /dev/null and b/docs/gitbook/resources/images/randamplify_elapsed.png differ

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/52bc44fa/docs/gitbook/tips/emr.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/tips/emr.md b/docs/gitbook/tips/emr.md
index 030a594..61cb25b 100644
--- a/docs/gitbook/tips/emr.md
+++ b/docs/gitbook/tips/emr.md
@@ -44,8 +44,10 @@ I'm usually lunching EMR instances with cheap Spot instances through [CLI client
 _To use YARN instead of old Hadoop, specify "[--ami-version 3.0.0](http://docs.aws.amazon.com/ElasticMapReduce/latest/DeveloperGuide/emr-plan-ami.html#ami-versions-supported)". Hivemall works on both old Hadoop and YARN._
 
 Or, lunch an interactive EMR job using the EMR GUI wizard.
-![emr-wizard](https://dl.dropboxusercontent.com/u/13123103/emr-wizard.png)
-![emr-bootstrap](https://dl.dropboxusercontent.com/u/13123103/emr-bootstrap.png)
+
+<img src="../resources/images/emr-wizard.png" alt="emr-wizard"/>
+
+<img src="../resources/images/emr-bootstrap.png" alt="emr-bootstrap"/>
 
 ## Data preparation
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/52bc44fa/docs/gitbook/tips/rand_amplify.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/tips/rand_amplify.md b/docs/gitbook/tips/rand_amplify.md
index 55b8caf..cd546ec 100644
--- a/docs/gitbook/tips/rand_amplify.md
+++ b/docs/gitbook/tips/rand_amplify.md
@@ -70,8 +70,7 @@ group by feature;
 ```
 
 The above query is executed by 2 MapReduce jobs as shown below:
-![amplifier](https://dl.dropboxusercontent.com/u/13123103/hivemall/amplify.png)
-[Here](https://dl.dropboxusercontent.com/u/13123103/hivemall/amplify_plan.txt) is the actual plan generated by the Hive.
+<img src="../resources/images/amplify.png" alt="amplifier"/>
 
 Using *trainning_x3*  instead of the plain training table results in higher and better AUC (0.746214) in [this](https://github.com/myui/hivemall/wiki/KDDCup-2012-track-2-CTR-prediction-(regression\)) example.
 
@@ -80,7 +79,7 @@ When the training table is so large that involves 100 Map tasks, the merge opera
 
 Note that the actual bottleneck is not M/R iterations but shuffling training instance. Iteration without shuffling (as in [the Spark example](http://spark.incubator.apache.org/examples.html)) causes very slow convergence and results in requiring more iterations. Shuffling cannot be avoided even in iterative MapReduce variants.
 
-![amplify elapsed](https://dl.dropboxusercontent.com/u/13123103/hivemall/amplify_elapsed.png)
+<img src="../resources/images/amplify_elapsed.png" alt="amplify_elapsed"/>
 
 ---
 # Amplify and shuffle training examples in each Map task
@@ -101,12 +100,12 @@ from
 ```
 
 The training query is executed as follows:
-![Random amplify](https://dl.dropboxusercontent.com/u/13123103/hivemall/randamplify.png)  
-[Here](https://dl.dropboxusercontent.com/u/13123103/hivemall/randamplify_plan.txt) is the actual query plan.
+
+<img src="../resources/images/randamplify.png" alt="randamplify"/>
 
 The map-local multiplication and shuffling has no bottleneck in the merge phase and the query is efficiently executed within a single MapReduce job.
 
-![rand_amplify elapsed ](https://dl.dropboxusercontent.com/u/13123103/hivemall/randamplify_elapsed.png)
+<img src="../resources/images/randamplify_elapsed.png" alt="randamplify_elapsed"/>
 
 Using *rand_amplify* results in a better AUC (0.743392) in [this](https://github.com/myui/hivemall/wiki/KDDCup-2012-track-2-CTR-prediction-(regression\)) example.
 


[28/50] [abbrv] incubator-hivemall git commit: Fixed to run mvn license:check

Posted by my...@apache.org.
Fixed to run mvn license:check

Project: http://git-wip-us.apache.org/repos/asf/incubator-hivemall/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-hivemall/commit/02b839b4
Tree: http://git-wip-us.apache.org/repos/asf/incubator-hivemall/tree/02b839b4
Diff: http://git-wip-us.apache.org/repos/asf/incubator-hivemall/diff/02b839b4

Branch: refs/heads/master
Commit: 02b839b4f2171f9b9a47437b7d0ad12e477b70e2
Parents: 6770098
Author: myui <yu...@gmail.com>
Authored: Mon Nov 7 15:20:02 2016 +0900
Committer: myui <yu...@gmail.com>
Committed: Mon Nov 7 15:20:02 2016 +0900

----------------------------------------------------------------------
 pom.xml                         | 14 ++++++++++++--
 resources/header-definition.xml |  3 +--
 2 files changed, 13 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/02b839b4/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 62cc5a5..094f034 100644
--- a/pom.xml
+++ b/pom.xml
@@ -391,7 +391,6 @@
 							<currentYear>${build.year}</currentYear>
 							<copyrightOwner>${project.organization.name}</copyrightOwner>
 						</properties>
-						<useDefaultExcludes>false</useDefaultExcludes>
 						<includes>
 							<include>src/main/**/*.java</include>
 							<include>src/test/**/*.java</include>
@@ -400,14 +399,25 @@
 							<include>bin/*.sh</include>
 							<include>docs/gitbook/**/*.md</include>
 						</includes>
+						<useDefaultExcludes>false</useDefaultExcludes>
+						<excludes>
+							<exclude>docs/gitbook/node_modules/**</exclude>
+						</excludes>
 						<encoding>UTF-8</encoding>
 						<headerDefinitions>
 							<headerDefinition>${main.basedir}/resources/header-definition.xml</headerDefinition>
 						</headerDefinitions>
 						<mapping>
-						  <md>XML_STYLE</md>
+							<md>XML_STYLE</md>
 						</mapping>
 					</configuration>
+					<executions>
+						<execution>
+							<goals>
+								<goal>check</goal>
+							</goals>
+						</execution>
+					</executions>
 				</plugin>
 			</plugins>
 		</pluginManagement>

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/02b839b4/resources/header-definition.xml
----------------------------------------------------------------------
diff --git a/resources/header-definition.xml b/resources/header-definition.xml
index 232b1d7..e28f8f8 100644
--- a/resources/header-definition.xml
+++ b/resources/header-definition.xml
@@ -14,8 +14,7 @@
     <xml_style>
         <firstLine><![CDATA[<!--]]></firstLine>
         <beforeEachLine>  </beforeEachLine>
-        <endLine><![CDATA[-->
-        ]]></endLine>
+        <endLine><![CDATA[-->]]></endLine>
         <skipLine><![CDATA[^<\?xml.*>$]]></skipLine>
         <firstLineDetectionPattern><![CDATA[(\s|\t)*<!--.*$]]></firstLineDetectionPattern>
         <lastLineDetectionPattern><![CDATA[.*-->(\s|\t)*$]]></lastLineDetectionPattern>


[04/50] [abbrv] incubator-hivemall git commit: Added a gitbook userguide

Posted by my...@apache.org.
Added a gitbook userguide

Project: http://git-wip-us.apache.org/repos/asf/incubator-hivemall/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-hivemall/commit/370e2aa3
Tree: http://git-wip-us.apache.org/repos/asf/incubator-hivemall/tree/370e2aa3
Diff: http://git-wip-us.apache.org/repos/asf/incubator-hivemall/diff/370e2aa3

Branch: refs/heads/master
Commit: 370e2aa339bc5ee5cdb95e40f4e4e31cd51283da
Parents: d3d7b55
Author: myui <yu...@gmail.com>
Authored: Sun Oct 23 03:44:52 2016 +0900
Committer: myui <yu...@gmail.com>
Committed: Sun Oct 23 04:11:23 2016 +0900

----------------------------------------------------------------------
 docs/gitbook/.gitignore                         |   3 +
 docs/gitbook/README.md                          |  22 +
 docs/gitbook/SUMMARY.md                         | 129 +++++
 docs/gitbook/anomaly/lof.md                     | 185 ++++++
 docs/gitbook/binaryclass/a9a.md                 |   0
 docs/gitbook/binaryclass/a9a_dataset.md         |  43 ++
 docs/gitbook/binaryclass/a9a_lr.md              |  79 +++
 docs/gitbook/binaryclass/a9a_minibatch.md       |  37 ++
 docs/gitbook/binaryclass/kdd2010a.md            |   0
 docs/gitbook/binaryclass/kdd2010a_dataset.md    |  72 +++
 docs/gitbook/binaryclass/kdd2010a_scw.md        | 185 ++++++
 docs/gitbook/binaryclass/kdd2010b.md            |   0
 docs/gitbook/binaryclass/kdd2010b_arow.md       |  51 ++
 docs/gitbook/binaryclass/kdd2010b_dataset.md    |  59 ++
 docs/gitbook/binaryclass/news20.md              |   0
 docs/gitbook/binaryclass/news20_adagrad.md      | 170 ++++++
 docs/gitbook/binaryclass/news20_dataset.md      |  84 +++
 docs/gitbook/binaryclass/news20_pa.md           | 243 ++++++++
 docs/gitbook/binaryclass/news20_scw.md          | 272 +++++++++
 docs/gitbook/binaryclass/webspam.md             |   0
 docs/gitbook/binaryclass/webspam_dataset.md     |  76 +++
 docs/gitbook/binaryclass/webspam_scw.md         | 136 +++++
 docs/gitbook/book.json                          |  64 +++
 docs/gitbook/eval/datagen.md                    |   0
 docs/gitbook/eval/lr_datagen.md                 |  92 +++
 docs/gitbook/eval/stat_eval.md                  |  56 ++
 docs/gitbook/ft_engineering/ft_trans.md         |   0
 docs/gitbook/ft_engineering/hashing.md          | 103 ++++
 docs/gitbook/ft_engineering/quantify.md         | 164 ++++++
 docs/gitbook/ft_engineering/scaling.md          | 173 ++++++
 docs/gitbook/ft_engineering/tfidf.md            | 149 +++++
 docs/gitbook/ft_engineering/vectorizer.md       |  42 ++
 docs/gitbook/getting_started/README.md          |   1 +
 docs/gitbook/getting_started/input-format.md    | 214 +++++++
 docs/gitbook/getting_started/installation.md    |  25 +
 .../getting_started/permanent-functions.md      |  42 ++
 docs/gitbook/misc/generic_funcs.md              | 210 +++++++
 docs/gitbook/misc/tokenizer.md                  |  30 +
 docs/gitbook/misc/topk.md                       | 288 ++++++++++
 docs/gitbook/multiclass/iris.md                 |   0
 docs/gitbook/multiclass/iris_dataset.md         | 203 +++++++
 docs/gitbook/multiclass/iris_randomforest.md    | 307 ++++++++++
 docs/gitbook/multiclass/iris_scw.md             | 307 ++++++++++
 docs/gitbook/multiclass/news20.md               |   0
 docs/gitbook/multiclass/news20_dataset.md       |  77 +++
 docs/gitbook/multiclass/news20_ensemble.md      | 180 ++++++
 .../multiclass/news20_one-vs-the-rest.md        | 330 +++++++++++
 .../news20_one-vs-the-rest_dataset.md           |  52 ++
 docs/gitbook/multiclass/news20_pa.md            |  90 +++
 docs/gitbook/multiclass/news20_scw.md           | 319 +++++++++++
 docs/gitbook/pig/.gitkeep                       |   0
 docs/gitbook/recommend/cf.md                    |   0
 docs/gitbook/recommend/item_based_cf.md         | 565 +++++++++++++++++++
 docs/gitbook/recommend/movielens.md             |   0
 docs/gitbook/recommend/movielens_cv.md          |  63 +++
 docs/gitbook/recommend/movielens_dataset.md     | 160 ++++++
 docs/gitbook/recommend/movielens_fm.md          | 249 ++++++++
 docs/gitbook/recommend/movielens_mf.md          | 137 +++++
 docs/gitbook/recommend/news20.md                |   0
 docs/gitbook/recommend/news20_bbit_minhash.md   |  50 ++
 docs/gitbook/recommend/news20_jaccard.md        | 123 ++++
 docs/gitbook/recommend/news20_knn.md            | 103 ++++
 docs/gitbook/regression/e2006.md                |   0
 docs/gitbook/regression/e2006_arow.md           | 259 +++++++++
 docs/gitbook/regression/e2006_dataset.md        |  72 +++
 docs/gitbook/regression/kddcup12tr2.md          |   0
 docs/gitbook/regression/kddcup12tr2_adagrad.md  | 109 ++++
 docs/gitbook/regression/kddcup12tr2_dataset.md  | 227 ++++++++
 docs/gitbook/regression/kddcup12tr2_lr.md       | 141 +++++
 .../regression/kddcup12tr2_lr_amplify.md        | 103 ++++
 .../images/hivemall-logo-color-small.png        | Bin 0 -> 33630 bytes
 docs/gitbook/resources/images/techstack.png     | Bin 0 -> 198979 bytes
 docs/gitbook/spark/.gitkeep                     |   0
 docs/gitbook/tips/README.md                     |   0
 docs/gitbook/tips/addbias.md                    |  43 ++
 docs/gitbook/tips/emr.md                        | 182 ++++++
 docs/gitbook/tips/ensemble_learning.md          | 180 ++++++
 docs/gitbook/tips/general_tips.md               |   0
 docs/gitbook/tips/hadoop_tuning.md              |  79 +++
 docs/gitbook/tips/mixserver.md                  |  68 +++
 docs/gitbook/tips/rand_amplify.md               | 103 ++++
 docs/gitbook/tips/rowid.md                      |  31 +
 docs/gitbook/tips/rt_prediction.md              | 234 ++++++++
 docs/gitbook/troubleshooting/README.md          |   0
 docs/gitbook/troubleshooting/asterisk.md        |   3 +
 .../troubleshooting/mapjoin_classcastex.md      |   8 +
 .../troubleshooting/mapjoin_task_error.md       |   8 +
 docs/gitbook/troubleshooting/num_mappers.md     |  20 +
 docs/gitbook/troubleshooting/oom.md             |  20 +
 src/site/site.xml                               |   6 +-
 src/site/xdoc/index.xml.vm                      |   7 +-
 91 files changed, 8711 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/370e2aa3/docs/gitbook/.gitignore
----------------------------------------------------------------------
diff --git a/docs/gitbook/.gitignore b/docs/gitbook/.gitignore
new file mode 100644
index 0000000..b12433d
--- /dev/null
+++ b/docs/gitbook/.gitignore
@@ -0,0 +1,3 @@
+_book/
+node_modules/
+*.log

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/370e2aa3/docs/gitbook/README.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/README.md b/docs/gitbook/README.md
new file mode 100644
index 0000000..82602f8
--- /dev/null
+++ b/docs/gitbook/README.md
@@ -0,0 +1,22 @@
+# Introduction
+
+<div class="alert alert-info">
+Apache Hivemall is a collection of machine learning algorithms and versatile data analytics functions. It provides a number of ease of use machine learning functionalities through the <a href="https://cwiki.apache.org/confluence/display/Hive/LanguageManual+UDF">Apache Hive UDF/UDAF/UDTF interface</a>.
+</div>
+
+<div style="text-align:center"><img src="resources/images/hivemall-logo-color-small.png"/></div>
+
+Apache Hivemall offers a variety of functionalities: <strong>regression, classification, recommendation, anomaly detection, k-nearest neighbor, and feature engineering</strong>. It also supports state-of-the-art machine learning algorithms such as Soft Confidence Weighted, Adaptive Regularization of Weight Vectors, Factorization Machines, and AdaDelta. 
+
+## Architecture
+
+Apache Hivemall is mainly designed to run on [Apache Hive](https://hive.apache.org/) but it also supports [Apache Pig](https://pig.apache.org/) and [Apache Spark](http://spark.apache.org/) for the runtime.
+Thus, it can be considered as a cross platform library for machine learning; prediction models built by a batch query of Apache Hive can be used on Apache Spark/Pig, and conversely, prediction models build by Apache Spark can be used from Apache Hive/Pig.
+
+<div style="text-align:center"><img src="resources/images/techstack.png" width="80%" height="80%"/></div>
+
+---
+
+<font color="gray">
+<sub>Apache Hivemall is an effort undergoing incubation at The Apache Software Foundation (ASF), sponsored by the <a href="http://incubator.apache.org/">Apache Incubator</a>.</sub>
+</font>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/370e2aa3/docs/gitbook/SUMMARY.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/SUMMARY.md b/docs/gitbook/SUMMARY.md
new file mode 100644
index 0000000..d85f952
--- /dev/null
+++ b/docs/gitbook/SUMMARY.md
@@ -0,0 +1,129 @@
+# Summary
+
+## TABLE OF CONTENTS
+
+* [Getting Started](getting_started/README.md)
+    * [Installation](getting_started/installation.md)
+    * [Install as permanent functions](getting_started/permanent-functions.md) 
+    * [Input Format](getting_started/input-format.md)
+
+* [Tips for Effective Hivemall](tips/README.md)
+    * [Explicit addBias() for better prediction](tips/addbias.md)
+    * [Use rand_amplify() to better prediction results](tips/rand_amplify.md)
+    * [Real-time Prediction on RDBMS](tips/rt_prediction.md)
+    * [Ensemble learning for stable prediction](tips/ensemble_learning.md)
+    * [Mixing models for a better prediction convergence (MIX server)](tips/mixserver.md)
+    * [Run Hivemall on Amazon Elastic MapReduce](tips/emr.md)
+
+* [General Hive/Hadoop tips](tips/general_tips.md)
+    * [Adding rowid for each row](tips/rowid.md)
+    * [Hadoop tuning for Hivemall](tips/hadoop_tuning.md)
+
+* [Troubleshooting](troubleshooting/README.md)
+    * [OutOfMemoryError in training](troubleshooting/oom.md)
+    * [SemanticException Generate Map Join Task Error: Cannot serialize object](troubleshooting/mapjoin_task_error.md)
+    * [Asterisk argument for UDTF does not work](troubleshooting/asterisk.md)
+    * [The number of mappers is less than input splits in Hadoop 2.x](troubleshooting/num_mappers.md)
+    * [Map-side Join causes ClassCastException on Tez](troubleshooting/mapjoin_classcastex.md)
+
+## Part II - Generic Features
+
+* [List of generic Hivemall functions](misc/generic_funcs.md)
+* [Efficient Top-K query processing](misc/topk.md)
+* [English/Japanese Text Tokenizer](misc/tokenizer.md)
+
+## Part III - Feature Engineering
+
+* [Feature Scaling](ft_engineering/scaling.md)
+* [Feature Hashing](ft_engineering/hashing.md)
+* [TF-IDF calculation](ft_engineering/tfidf.md)
+
+* [FEATURE TRANSFORMATION](ft_engineering/ft_trans.md)
+    * [Vectorize Features](ft_engineering/vectorizer.md)
+    * [Quantify non-number features](ft_engineering/quantify.md)
+
+## Part IV - Evaluation
+
+* [Statistical evaluation of a prediction model](eval/stat_eval.md)
+
+* [Data Generation](eval/datagen.md)
+    * [Logistic Regression data generation](eval/lr_datagen.md)
+
+## Part V - Binary classification
+
+* [a9a Tutorial](binaryclass/a9a.md)
+    * [Data preparation](binaryclass/a9a_dataset.md)
+    * [Logistic Regression](binaryclass/a9a_lr.md)
+    * [Mini-batch Gradient Descent](binaryclass/a9a_minibatch.md)
+
+* [News20 Tutorial](binaryclass/news20.md)
+    * [Data preparation](binaryclass/news20_dataset.md)
+    * [Perceptron, Passive Aggressive](binaryclass/news20_pa.md)
+    * [CW, AROW, SCW](binaryclass/news20_scw.md)
+    * [AdaGradRDA, AdaGrad, AdaDelta](binaryclass/news20_adagrad.md)
+
+* [KDD2010a Tutorial](binaryclass/kdd2010a.md)
+    * [Data preparation](binaryclass/kdd2010a_dataset.md)
+    * [PA, CW, AROW, SCW](binaryclass/kdd2010a_scw.md)
+    
+* [KDD2010b Tutorial](binaryclass/kdd2010b.md)
+    * [Data preparation](binaryclass/kdd2010b_dataset.md)
+    * [AROW](binaryclass/kdd2010b_arow.md)
+
+* [Webspam Tutorial](binaryclass/webspam.md)
+    * [Data pareparation](binaryclass/webspam_dataset.md)
+    * [PA1, AROW, SCW](binaryclass/webspam_scw.md)
+    
+## Part VI - Multiclass classification
+
+* [News20 Multiclass Tutorial](multiclass/news20.md)
+    * [Data preparation](multiclass/news20_dataset.md)
+    * [Data preparation for one-vs-the-rest classifiers](multiclass/news20_one-vs-the-rest_dataset.md)
+    * [PA](multiclass/news20_pa.md)
+    * [CW, AROW, SCW](multiclass/news20_scw.md)
+    * [Ensemble learning](multiclass/news20_ensemble.md)
+    * [one-vs-the-rest classifier](multiclass/news20_one-vs-the-rest.md)
+    
+* [Iris Tutorial](multiclass/iris.md)
+    * [Data preparation](multiclass/iris_dataset.md)
+    * [SCW](multiclass/iris_scw.md)
+    * [RandomForest](multiclass/iris_randomforest.md)
+    
+## Part VII - Regression
+
+* [E2006-tfidf regression Tutorial](regression/e2006.md)
+    * [Data preparation](regression/e2006_dataset.md)
+    * [Passive Aggressive, AROW](regression/e2006_arow.md)
+
+* [KDDCup 2012 track 2 CTR prediction Tutorial](regression/kddcup12tr2.md)
+    * [Data preparation](regression/kddcup12tr2_dataset.md)
+    * [Logistic Regression, Passive Aggressive](regression/kddcup12tr2_lr.md)
+    * [Logistic Regression with Amplifier](regression/kddcup12tr2_lr_amplify.md)
+    * [AdaGrad, AdaDelta](regression/kddcup12tr2_adagrad.md)
+
+## Part VIII - Recommendation
+
+* [Collaborative Filtering](recommend/cf.md)
+    * [Item-based Collaborative Filtering](recommend/item_based_cf.md)
+
+* [News20 related article recommendation Tutorial](recommend/news20.md)
+    * [Data preparation](multiclass/news20_dataset.md)
+    * [LSH/Minhash and Jaccard Similarity](recommend/news20_jaccard.md)
+    * [LSH/Minhash and Brute-Force Search](recommend/news20_knn.md)
+    * [kNN search using b-Bits Minhash](recommend/news20_bbit_minhash.md)
+
+* [MovieLens movie recommendation Tutorial](recommend/movielens.md)
+    * [Data preparation](recommend/movielens_dataset.md)
+    * [Matrix Factorization](recommend/movielens_mf.md)
+    * [Factorization Machine](recommend/movielens_fm.md)
+    * [10-fold Cross Validation (Matrix Factorization)](recommend/movielens_cv.md)    
+
+## Part IX - Anomaly Detection
+
+* [Outlier Detection using Local Outlier Factor (LOF)](anomaly/lof.md)
+
+## Part X - External References
+
+* [Hivemall on Apache Spark](https://github.com/maropu/hivemall-spark)
+* [Hivemall on Apache Pig](https://github.com/daijyc/hivemall/wiki/PigHome)
+

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/370e2aa3/docs/gitbook/anomaly/lof.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/anomaly/lof.md b/docs/gitbook/anomaly/lof.md
new file mode 100644
index 0000000..f8f0b61
--- /dev/null
+++ b/docs/gitbook/anomaly/lof.md
@@ -0,0 +1,185 @@
+This article introduce how to find outliers using [Local Outlier Detection (LOF)](http://en.wikipedia.org/wiki/Local_outlier_factor) on Hivemall.
+
+# Data Preparation
+
+```sql
+create database lof;
+use lof;
+
+create external table hundred_balls (
+  rowid int, 
+  weight double,
+  specific_heat double,
+  reflectance double
+)
+ROW FORMAT DELIMITED
+  FIELDS TERMINATED BY ' '
+STORED AS TEXTFILE LOCATION '/dataset/lof/hundred_balls';
+```
+
+Download [hundred_balls.txt](https://github.com/myui/hivemall/blob/master/resources/examples/lof/hundred_balls.txt) that is originally provides in [this article](http://next.rikunabi.com/tech/docs/ct_s03600.jsp?p=002259).
+
+You can find outliers in [this picture](http://next.rikunabi.com/tech/contents/ts_report/img/201303/002259/part1_img1.jpg). As you can see, Rowid `87` is apparently an outlier.
+
+```sh
+awk '{FS=" "; OFS=" "; print NR,$0}' hundred_balls.txt | \
+hadoop fs -put - /dataset/lof/hundred_balls/hundred_balls.txt
+```
+
+```sql
+create table train
+as
+select rowid, array(concat("weight:", weight), concat("specific_heat:", specific_heat), concat("reflectance:", reflectance)) as features
+from hundred_balls;
+```
+
+## Apply Data Normalization
+
+```sql
+create table train_normalized
+as
+WITH fv as (
+select 
+  rowid, 
+  extract_feature(feature) as feature,
+  extract_weight(feature) as value
+from 
+  train 
+  LATERAL VIEW explode(features) exploded AS feature
+), 
+stats as (
+select
+  feature,
+  -- avg(value) as mean, stddev_pop(value) as stddev
+  min(value) as min, max(value) as max
+from
+  fv
+group by
+  feature
+), 
+norm as (
+select 
+  rowid, 
+  t1.feature, 
+  -- zscore(t1.value, t2.mean, t2.stddev) as zscore
+  rescale(t1.value, t2.min, t2.max) as minmax
+from 
+  fv t1 JOIN
+  stats t2 ON (t1.feature = t2.feature) 
+),
+norm_fv as (
+select
+  rowid, 
+  -- concat(feature, ":", zscore) as feature
+  concat(feature, ":", minmax) as feature
+from
+  norm
+)
+select 
+  rowid, 
+  collect_list(feature) as features
+from
+  norm_fv
+group by
+  rowid
+;
+```
+
+```
+hive> select * from train_normalized limit 3;
+
+1       ["reflectance:0.5252967","specific_heat:0.19863537","weight:0.0"]
+2       ["reflectance:0.5950446","specific_heat:0.09166764","weight:0.052084323"]
+3       ["reflectance:0.6797837","specific_heat:0.12567581","weight:0.13255163"]
+```
+
+# Outlier Detection using Local Outlier Facotor (LOF)
+
+```sql
+-- workaround to deal with a bug in Hive/Tez
+-- https://issues.apache.org/jira/browse/HIVE-10729
+-- set hive.auto.convert.join=false;
+set hive.mapjoin.optimized.hashtable=false;
+
+-- parameter of LoF
+set hivevar:k=12;
+
+-- find topk outliers
+set hivevar:topk=3;
+```
+
+```sql
+create table list_neighbours
+as
+select
+  each_top_k(
+    -${k}, t1.rowid, euclid_distance(t1.features, t2.features), 
+    t1.rowid, 
+    t2.rowid
+  ) as (rank, distance, target, neighbour)
+from 
+  train_normalized t1
+  LEFT OUTER JOIN train_normalized t2
+where
+  t1.rowid != t2.rowid
+;
+```
+
+_Note: `list_neighbours` table SHOULD be created because `list_neighbours` is used multiple times._
+
+_Note: [`each_top_k`](https://github.com/myui/hivemall/pull/196) is supported from Hivemall v0.3.2-3 or later._
+
+_Note: To parallelize a top-k computation, break LEFT-hand table into piece as describe in [this page](https://github.com/myui/hivemall/wiki/Efficient-Top-k-computation-on-Apache-Hive-using-Hivemall-UDTF#parallelization-of-similarity-computation-using-with-clause)._
+
+```sql
+WITH k_distance as (
+select
+  target, 
+  max(distance) as k_distance
+from
+  list_neighbours
+group by
+  target
+), 
+reach_distance as (
+select
+  t1.target,
+  max2(t2.k_distance, t1.distance) as reach_distance
+from
+  list_neighbours t1 JOIN 
+  k_distance t2 ON (t1.neighbour = t2.target)
+), 
+lrd as (
+select
+  target,   
+  1.0 / avg(reach_distance) as lrd
+from
+  reach_distance
+group by
+  target
+), 
+neighbours_lrd as (
+select
+  t1.target, 
+  t2.lrd
+from
+  list_neighbours t1 JOIN
+  lrd t2 on (t1.neighbour = t2.target)
+)
+select
+  t1.target, 
+  sum(t2.lrd / t1.lrd) / count(1) as lof
+from
+  lrd t1 JOIN
+  neighbours_lrd t2 on (t1.target = t2.target)
+group by
+  t1.target
+order by lof desc
+limit ${topk};
+```
+
+```
+> 87      3.031143749957831
+> 16      1.9755564408378874
+> 1       1.8415763570939774
+```

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/370e2aa3/docs/gitbook/binaryclass/a9a.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/binaryclass/a9a.md b/docs/gitbook/binaryclass/a9a.md
new file mode 100644
index 0000000..e69de29

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/370e2aa3/docs/gitbook/binaryclass/a9a_dataset.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/binaryclass/a9a_dataset.md b/docs/gitbook/binaryclass/a9a_dataset.md
new file mode 100644
index 0000000..28bcd57
--- /dev/null
+++ b/docs/gitbook/binaryclass/a9a_dataset.md
@@ -0,0 +1,43 @@
+a9a
+===
+http://www.csie.ntu.edu.tw/~cjlin/libsvmtools/datasets/binary.html#a9a
+
+---
+
+preparation
+=========
+
+[conv.awk](https://raw.githubusercontent.com/myui/hivemall/master/resources/misc/conv.awk)
+
+```
+cd /mnt/archive/datasets/classification/a9a
+awk -f conv.awk a9a | sed -e "s/+1/1/" | sed -e "s/-1/0/" > a9a.train
+awk -f conv.awk a9a.t | sed -e "s/+1/1/" | sed -e "s/-1/0/" > a9a.test
+```
+
+## Putting data on HDFS
+```
+hadoop fs -mkdir -p /dataset/a9a/train
+hadoop fs -mkdir -p /dataset/a9a/test
+
+hadoop fs -copyFromLocal a9a.train /dataset/a9a/train
+hadoop fs -copyFromLocal a9a.test /dataset/a9a/test
+```
+
+## Training/test data prepareation
+```sql
+create database a9a;
+use a9a;
+
+create external table a9atrain (
+  rowid int,
+  label float,
+  features ARRAY<STRING>
+) ROW FORMAT DELIMITED FIELDS TERMINATED BY '\t' COLLECTION ITEMS TERMINATED BY "," STORED AS TEXTFILE LOCATION '/dataset/a9a/train';
+
+create external table a9atest (
+  rowid int, 
+  label float,
+  features ARRAY<STRING>
+) ROW FORMAT DELIMITED FIELDS TERMINATED BY '\t' COLLECTION ITEMS TERMINATED BY "," STORED AS TEXTFILE LOCATION '/dataset/a9a/test';
+```
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/370e2aa3/docs/gitbook/binaryclass/a9a_lr.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/binaryclass/a9a_lr.md b/docs/gitbook/binaryclass/a9a_lr.md
new file mode 100644
index 0000000..5029c49
--- /dev/null
+++ b/docs/gitbook/binaryclass/a9a_lr.md
@@ -0,0 +1,79 @@
+a9a
+===
+http://www.csie.ntu.edu.tw/~cjlin/libsvmtools/datasets/binary.html#a9a
+
+_Training with iterations is OBSOLUTE in Hivemall._  
+_Using amplifier and shuffling inputs is RECOMMENDED in Hivemall._
+
+---
+
+## UDF preparation
+
+```sql
+select count(1) from a9atrain;
+-- set total_steps ideally be "count(1) / #map tasks"
+set hivevar:total_steps=32561;
+
+select count(1) from a9atest;
+set hivevar:num_test_instances=16281;
+```
+
+## training
+```sql
+create table a9a_model1 
+as
+select 
+ cast(feature as int) as feature,
+ avg(weight) as weight
+from 
+ (select 
+     logress(addBias(features),label,"-total_steps ${total_steps}") as (feature,weight)
+  from 
+     a9atrain
+ ) t 
+group by feature;
+```
+_"-total_steps" option is optional for logress() function._  
+_I recommend you NOT to use options (e.g., total_steps and eta0) if you are not familiar with those options. Hivemall then uses an autonomic ETA (learning rate) estimator._
+
+## prediction
+```sql
+create or replace view a9a_predict1 
+as
+WITH a9atest_exploded as (
+select 
+  rowid,
+  label,
+  extract_feature(feature) as feature,
+  extract_weight(feature) as value
+from 
+  a9atest LATERAL VIEW explode(addBias(features)) t AS feature
+)
+select
+  t.rowid, 
+  sigmoid(sum(m.weight * t.value)) as prob,
+  CAST((case when sigmoid(sum(m.weight * t.value)) >= 0.5 then 1.0 else 0.0 end) as FLOAT) as label
+from 
+  a9atest_exploded t LEFT OUTER JOIN
+  a9a_model1 m ON (t.feature = m.feature)
+group by
+  t.rowid;
+```
+
+## evaluation
+```sql
+create or replace view a9a_submit1 as
+select 
+  t.label as actual, 
+  pd.label as predicted, 
+  pd.prob as probability
+from 
+  a9atest t JOIN a9a_predict1 pd 
+    on (t.rowid = pd.rowid);
+```
+
+```sql
+select count(1) / ${num_test_instances} from a9a_submit1 
+where actual == predicted;
+```
+> 0.8430071862907684
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/370e2aa3/docs/gitbook/binaryclass/a9a_minibatch.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/binaryclass/a9a_minibatch.md b/docs/gitbook/binaryclass/a9a_minibatch.md
new file mode 100644
index 0000000..714db6a
--- /dev/null
+++ b/docs/gitbook/binaryclass/a9a_minibatch.md
@@ -0,0 +1,37 @@
+This page explains how to apply [Mini-Batch Gradient Descent](https://class.coursera.org/ml-003/lecture/106) for the training of logistic regression explained in [this example](https://github.com/myui/hivemall/wiki/a9a-binary-classification-(logistic-regression)). 
+
+See [this page](https://github.com/myui/hivemall/wiki/a9a-binary-classification-(logistic-regression)) first. This content depends on it.
+
+# Training
+
+Replace `a9a_model1` of [this example](https://github.com/myui/hivemall/wiki/a9a-binary-classification-(logistic-regression)).
+
+```sql
+set hivevar:total_steps=32561;
+set hivevar:mini_batch_size=10;
+
+create table a9a_model1 
+as
+select 
+ cast(feature as int) as feature,
+ avg(weight) as weight
+from 
+ (select 
+     logress(addBias(features),label,"-total_steps ${total_steps} -mini_batch ${mini_batch_size}") as (feature,weight)
+  from 
+     a9atrain
+ ) t 
+group by feature;
+```
+
+# Evaluation
+
+```sql
+select count(1) / ${num_test_instances} from a9a_submit1 
+where actual == predicted;
+```
+
+
+| Stochastic Gradient Descent | Minibatch Gradient Descent |
+| ------------- | ------------- |
+| 0.8430071862907684 | 0.8463239358761747 |
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/370e2aa3/docs/gitbook/binaryclass/kdd2010a.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/binaryclass/kdd2010a.md b/docs/gitbook/binaryclass/kdd2010a.md
new file mode 100644
index 0000000..e69de29

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/370e2aa3/docs/gitbook/binaryclass/kdd2010a_dataset.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/binaryclass/kdd2010a_dataset.md b/docs/gitbook/binaryclass/kdd2010a_dataset.md
new file mode 100644
index 0000000..731d68b
--- /dev/null
+++ b/docs/gitbook/binaryclass/kdd2010a_dataset.md
@@ -0,0 +1,72 @@
+[http://www.csie.ntu.edu.tw/~cjlin/libsvmtools/datasets/binary.html#kdd2010 (algebra)](http://www.csie.ntu.edu.tw/~cjlin/libsvmtools/datasets/binary.html#kdd2010 (algebra))
+
+* # of classes: 2
+* # of data: 8,407,752 (training) / 510,302 (testing)
+* # of features: 20,216,830 in about 2.73 GB (training) / 20,216,830 (testing) 
+
+---
+# Define training/testing tables
+```sql
+add jar ./tmp/hivemall.jar;
+source ./tmp/define-all.hive;
+
+create database kdd2010;
+use kdd2010;
+
+create external table kdd10a_train (
+  rowid int,
+  label int,
+  features ARRAY<STRING>
+) 
+ROW FORMAT DELIMITED FIELDS TERMINATED BY '\t' COLLECTION ITEMS TERMINATED BY "," 
+STORED AS TEXTFILE LOCATION '/dataset/kdd10a/train';
+
+create external table kdd10a_test (
+  rowid int, 
+  label int,
+  features ARRAY<STRING>
+) 
+ROW FORMAT DELIMITED FIELDS TERMINATED BY '\t' COLLECTION ITEMS TERMINATED BY "," 
+STORED AS TEXTFILE LOCATION '/dataset/kdd10a/test';
+```
+
+# Putting data into HDFS
+[conv.awk](https://raw.githubusercontent.com/myui/hivemall/master/scripts/misc/conv.awk)
+```sh
+awk -f conv.awk kdda | hadoop fs -put - /dataset/kdd10a/train/kdda
+awk -f conv.awk kdda.t | hadoop fs -put - /dataset/kdd10a/test/kdda.t
+```
+
+# Make auxiliary tables
+```sql
+create table kdd10a_train_orcfile (
+ rowid bigint,
+ label int,
+ features array<string>
+) STORED AS orc tblproperties ("orc.compress"="SNAPPY");
+
+-- SET mapred.reduce.tasks=64;
+INSERT OVERWRITE TABLE kdd10a_train_orcfile
+select * from kdd10a_train
+CLUSTER BY rand();
+-- SET mapred.reduce.tasks=-1;
+
+create table kdd10a_test_exploded as
+select 
+  rowid,
+  label,
+  split(feature,":")[0] as feature,
+  cast(split(feature,":")[1] as float) as value
+from 
+  kdd10a_test LATERAL VIEW explode(addBias(features)) t AS feature;
+
+set hivevar:xtimes=3;
+set hivevar:shufflebuffersize=1000;
+-- set hivemall.amplify.seed=32;
+create or replace view kdd10a_train_x3
+as
+select
+   rand_amplify(${xtimes}, ${shufflebuffersize}, *) as (rowid, label, features)
+from  
+   kdd10a_train_orcfile;
+```
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/370e2aa3/docs/gitbook/binaryclass/kdd2010a_scw.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/binaryclass/kdd2010a_scw.md b/docs/gitbook/binaryclass/kdd2010a_scw.md
new file mode 100644
index 0000000..ee8fbba
--- /dev/null
+++ b/docs/gitbook/binaryclass/kdd2010a_scw.md
@@ -0,0 +1,185 @@
+# PA1
+## Train
+```sql
+-- SET mapred.reduce.tasks=32;
+drop table kdd10a_pa1_model1;
+create table kdd10a_pa1_model1 as
+select 
+ feature,
+ voted_avg(weight) as weight
+from 
+ (select 
+     train_pa1(addBias(features),label) as (feature,weight)
+  from 
+     kdd10a_train_x3
+ ) t 
+group by feature;
+```
+
+## Predict
+```sql
+create or replace view kdd10a_pa1_predict1 
+as
+select
+  t.rowid, 
+  sum(m.weight * t.value) as total_weight,
+  case when sum(m.weight * t.value) > 0.0 then 1 else -1 end as label
+from 
+  kdd10a_test_exploded t LEFT OUTER JOIN
+  kdd10a_pa1_model1 m ON (t.feature = m.feature)
+group by
+  t.rowid;
+```
+
+# Evaluate
+```sql
+create or replace view kdd10a_pa1_submit1 as
+select 
+  t.rowid, 
+  t.label as actual, 
+  pd.label as predicted
+from 
+  kdd10a_test t JOIN kdd10a_pa1_predict1 pd 
+    on (t.rowid = pd.rowid);
+
+select count(1)/510302 from kdd10a_pa1_submit1 
+where actual = predicted;
+```
+> 0.8677782959894337
+
+# CW
+```sql
+-- SET mapred.reduce.tasks=32;
+drop table kdd10a_cw_model1;
+create table kdd10a_cw_model1 as
+select 
+ feature,
+ argmin_kld(weight, covar) as weight
+from 
+ (select 
+     train_cw(addBias(features),label) as (feature,weight,covar)
+  from 
+     kdd10a_train_x3
+ ) t 
+group by feature;
+
+create or replace view kdd10a_cw_predict1 
+as
+select
+  t.rowid, 
+  sum(m.weight * t.value) as total_weight,
+  case when sum(m.weight * t.value) > 0.0 then 1 else -1 end as label
+from 
+  kdd10a_test_exploded t LEFT OUTER JOIN
+  kdd10a_cw_model1 m ON (t.feature = m.feature)
+group by
+  t.rowid;
+
+create or replace view kdd10a_cw_submit1 as
+select 
+  t.rowid, 
+  t.label as actual, 
+  pd.label as predicted
+from 
+  kdd10a_test t JOIN kdd10a_cw_predict1 pd 
+    on (t.rowid = pd.rowid);
+
+select count(1)/510302 from kdd10a_cw_submit1 
+where actual = predicted;
+```
+> 0.8678037711002504
+
+# AROW
+```sql
+-- SET mapred.reduce.tasks=32;
+drop table kdd10a_arow_model1;
+create table kdd10a_arow_model1 as
+select 
+ feature,
+ -- voted_avg(weight) as weight
+ argmin_kld(weight, covar) as weight -- [hivemall v0.2alpha3 or later]
+from 
+ (select 
+     -- train_arow(addBias(features),label) as (feature,weight) -- [hivemall v0.1]
+     train_arow(addBias(features),label) as (feature,weight,covar) -- [hivemall v0.2 or later]
+  from 
+     kdd10a_train_x3
+ ) t 
+group by feature;
+
+create or replace view kdd10a_arow_predict1 
+as
+select
+  t.rowid, 
+  sum(m.weight * t.value) as total_weight,
+  case when sum(m.weight * t.value) > 0.0 then 1 else -1 end as label
+from 
+  kdd10a_test_exploded t LEFT OUTER JOIN
+  kdd10a_arow_model1 m ON (t.feature = m.feature)
+group by
+  t.rowid;
+
+create or replace view kdd10a_arow_submit1 as
+select 
+  t.rowid, 
+  t.label as actual, 
+  pd.label as predicted
+from 
+  kdd10a_test t JOIN kdd10a_arow_predict1 pd 
+    on (t.rowid = pd.rowid);
+
+select count(1)/510302 from kdd10a_arow_submit1 
+where actual = predicted;
+```
+> 0.8676038894615345
+
+# SCW
+```sql
+-- SET mapred.reduce.tasks=32;
+drop table kdd10a_scw_model1;
+create table kdd10a_scw_model1 as
+select 
+ feature,
+ argmin_kld(weight, covar) as weight
+from 
+ (select 
+     train_scw(addBias(features),label) as (feature,weight,covar)
+  from 
+     kdd10a_train_x3
+ ) t 
+group by feature;
+
+create or replace view kdd10a_scw_predict1 
+as
+select
+  t.rowid, 
+  sum(m.weight * t.value) as total_weight,
+  case when sum(m.weight * t.value) > 0.0 then 1 else -1 end as label
+from 
+  kdd10a_test_exploded t LEFT OUTER JOIN
+  kdd10a_scw_model1 m ON (t.feature = m.feature)
+group by
+  t.rowid;
+
+create or replace view kdd10a_scw_submit1 as
+select 
+  t.rowid, 
+  t.label as actual, 
+  pd.label as predicted
+from 
+  kdd10a_test t JOIN kdd10a_scw_predict1 pd 
+    on (t.rowid = pd.rowid);
+
+select count(1)/510302 from kdd10a_scw_submit1 
+where actual = predicted;
+```
+> 0.8678096499719774
+
+---
+
+| Algorithm | Accuracy |
+|:-----------|------------:|
+| AROW | 0.8676038894615345 |
+| PA1 | 0.8677782959894337 |
+| CW | 0.8678037711002504 |
+| SCW1 | 0.8678096499719774 |
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/370e2aa3/docs/gitbook/binaryclass/kdd2010b.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/binaryclass/kdd2010b.md b/docs/gitbook/binaryclass/kdd2010b.md
new file mode 100644
index 0000000..e69de29

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/370e2aa3/docs/gitbook/binaryclass/kdd2010b_arow.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/binaryclass/kdd2010b_arow.md b/docs/gitbook/binaryclass/kdd2010b_arow.md
new file mode 100644
index 0000000..7ac845a
--- /dev/null
+++ b/docs/gitbook/binaryclass/kdd2010b_arow.md
@@ -0,0 +1,51 @@
+## training
+```sql
+-- SET mapred.reduce.tasks=32;
+drop table kdd10b_arow_model1;
+create table kdd10b_arow_model1 as
+select 
+ feature,
+ -- voted_avg(weight) as weight
+ argmin_kld(weight, covar) as weight -- [hivemall v0.2alpha3 or later]
+from 
+ (select 
+     -- train_arow(addBias(features),label) as (feature,weight) -- [hivemall v0.1]
+     train_arow(addBias(features),label) as (feature,weight,covar) -- [hivemall v0.2 or later]
+  from 
+     kdd10b_train_x3
+ ) t 
+group by feature;
+```
+
+## prediction
+```sql
+create or replace view kdd10b_arow_predict1 
+as
+select
+  t.rowid, 
+  sum(m.weight * t.value) as total_weight,
+  case when sum(m.weight * t.value) > 0.0 then 1 else -1 end as label
+from 
+  kdd10b_test_exploded t LEFT OUTER JOIN
+  kdd10b_arow_model1 m ON (t.feature = m.feature)
+group by
+  t.rowid;
+```
+
+## evaluation
+```sql
+create or replace view kdd10b_arow_submit1 as
+select 
+  t.rowid, 
+  t.label as actual, 
+  pd.label as predicted
+from 
+  kdd10b_test t JOIN kdd10b_arow_predict1 pd 
+    on (t.rowid = pd.rowid);
+```
+
+```sql
+select count(1)/748401 from kdd10b_arow_submit1 
+where actual = predicted;
+```
+> 0.8565808971393678
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/370e2aa3/docs/gitbook/binaryclass/kdd2010b_dataset.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/binaryclass/kdd2010b_dataset.md b/docs/gitbook/binaryclass/kdd2010b_dataset.md
new file mode 100644
index 0000000..5e26dba
--- /dev/null
+++ b/docs/gitbook/binaryclass/kdd2010b_dataset.md
@@ -0,0 +1,59 @@
+[http://www.csie.ntu.edu.tw/~cjlin/libsvmtools/datasets/binary.html#kdd2010 (bridge to algebra)](http://www.csie.ntu.edu.tw/~cjlin/libsvmtools/datasets/binary.html#kdd2010 (bridge to algebra))
+
+* # of classes: 2
+* # of data: 19,264,097 / 748,401 (testing)
+* # of features: 29,890,095 / 29,890,095 (testing)
+
+---
+# Define training/testing tables
+```sql
+add jar ./tmp/hivemall.jar;
+source ./tmp/define-all.hive;
+
+create database kdd2010;
+use kdd2010;
+
+create external table kdd10b_train (
+  rowid int,
+  label int,
+  features ARRAY<STRING>
+) 
+ROW FORMAT DELIMITED FIELDS TERMINATED BY '\t' COLLECTION ITEMS TERMINATED BY "," 
+STORED AS TEXTFILE LOCATION '/dataset/kdd10b/train';
+
+create external table kdd10b_test (
+  rowid int, 
+  label int,
+  features ARRAY<STRING>
+) 
+ROW FORMAT DELIMITED FIELDS TERMINATED BY '\t' COLLECTION ITEMS TERMINATED BY "," 
+STORED AS TEXTFILE LOCATION '/dataset/kdd10b/test';
+```
+
+# Putting data into HDFS
+[conv.awk](https://raw.githubusercontent.com/myui/hivemall/master/scripts/misc/conv.awk)
+```sh
+awk -f conv.awk kddb | hadoop fs -put - /dataset/kdd10b/train/kddb
+awk -f conv.awk kddb.t | hadoop fs -put - /dataset/kdd10b/test/kddb.t
+```
+
+# Make auxiliary tables
+```sql
+create table kdd10b_test_exploded as
+select 
+  rowid,
+  label,
+  split(feature,":")[0] as feature,
+  cast(split(feature,":")[1] as float) as value
+from 
+  kdd10b_test LATERAL VIEW explode(addBias(features)) t AS feature;
+
+set hivevar:xtimes=3;
+set hivevar:shufflebuffersize=1000;
+create or replace view kdd10b_train_x3
+as
+select
+   rand_amplify(${xtimes}, ${shufflebuffersize}, *) as (rowid, label, features)
+from  
+   kdd10b_train;
+```

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/370e2aa3/docs/gitbook/binaryclass/news20.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/binaryclass/news20.md b/docs/gitbook/binaryclass/news20.md
new file mode 100644
index 0000000..e69de29

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/370e2aa3/docs/gitbook/binaryclass/news20_adagrad.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/binaryclass/news20_adagrad.md b/docs/gitbook/binaryclass/news20_adagrad.md
new file mode 100644
index 0000000..08e39df
--- /dev/null
+++ b/docs/gitbook/binaryclass/news20_adagrad.md
@@ -0,0 +1,170 @@
+_Note that this feature is supported since Hivemall v0.3-beta2 or later._
+
+## UDF preparation
+```
+add jar ./tmp/hivemall-with-dependencies.jar;
+source ./tmp/define-all.hive;
+
+use news20;
+```
+
+#[AdaGradRDA]
+
+_Note that the current AdaGradRDA implmenetation can only be applied to classification, not to regression, because it uses hinge loss for the loss function._
+
+
+## model building
+```sql
+drop table news20b_adagrad_rda_model1;
+create table news20b_adagrad_rda_model1 as
+select 
+ feature,
+ voted_avg(weight) as weight
+from 
+ (select 
+     train_adagrad_rda(addBias(features),label) as (feature,weight)
+  from 
+     news20b_train_x3
+ ) t 
+group by feature;
+```
+
+## prediction
+```sql
+create or replace view news20b_adagrad_rda_predict1 
+as
+select
+  t.rowid, 
+  sum(m.weight * t.value) as total_weight,
+  case when sum(m.weight * t.value) > 0.0 then 1 else -1 end as label
+from 
+  news20b_test_exploded t LEFT OUTER JOIN
+  news20b_adagrad_rda_model1 m ON (t.feature = m.feature)
+group by
+  t.rowid;
+```
+
+## evaluation
+```sql
+create or replace view news20b_adagrad_rda_submit1 as
+select 
+  t.label as actual, 
+  pd.label as predicted
+from 
+  news20b_test t JOIN news20b_adagrad_rda_predict1 pd 
+    on (t.rowid = pd.rowid);
+```
+
+```sql
+select count(1)/4996 from news20b_adagrad_rda_submit1 
+where actual == predicted;
+```
+> SCW1 0.9661729383506805 
+
+> ADAGRAD+RDA 0.9677742193755005
+
+#[AdaGrad]
+
+_Note that AdaGrad is better suited for a regression problem because the current implementation only support logistic loss._
+
+## model building
+```sql
+drop table news20b_adagrad_model1;
+create table news20b_adagrad_model1 as
+select 
+ feature,
+ voted_avg(weight) as weight
+from 
+ (select 
+     adagrad(addBias(features),convert_label(label)) as (feature,weight)
+  from 
+     news20b_train_x3
+ ) t 
+group by feature;
+```
+_adagrad takes 0/1 for a label value and convert_label(label) converts a label value from -1/+1 to 0/1._
+## prediction
+```sql
+create or replace view news20b_adagrad_predict1 
+as
+select
+  t.rowid, 
+  case when sigmoid(sum(m.weight * t.value)) >= 0.5 then 1 else -1 end as label
+from 
+  news20b_test_exploded t LEFT OUTER JOIN
+  news20b_adagrad_model1 m ON (t.feature = m.feature)
+group by
+  t.rowid;
+```
+
+## evaluation
+```sql
+create or replace view news20b_adagrad_submit1 as
+select 
+  t.label as actual, 
+  p.label as predicted
+from 
+  news20b_test t JOIN news20b_adagrad_predict1 p
+    on (t.rowid = p.rowid);
+```
+
+```sql
+select count(1)/4996 from news20b_adagrad_submit1 
+where actual == predicted;
+```
+> 0.9549639711769415 (adagrad)
+
+#[AdaDelta]
+
+_Note that AdaDelta is better suited for regression problem because the current implementation only support logistic loss._
+
+## model building
+```sql
+drop table news20b_adadelta_model1;
+create table news20b_adadelta_model1 as
+select 
+ feature,
+ voted_avg(weight) as weight
+from 
+ (select 
+     adadelta(addBias(features),convert_label(label)) as (feature,weight)
+  from 
+     news20b_train_x3
+ ) t 
+group by feature;
+```
+
+## prediction
+```sql
+create or replace view news20b_adadelta_predict1 
+as
+select
+  t.rowid, 
+  case when sigmoid(sum(m.weight * t.value)) >= 0.5 then 1 else -1 end as label
+from 
+  news20b_test_exploded t LEFT OUTER JOIN
+  news20b_adadelta_model1 m ON (t.feature = m.feature)
+group by
+  t.rowid;
+```
+
+## evaluation
+```sql
+create or replace view news20b_adadelta_submit1 as
+select 
+  t.label as actual, 
+  p.label as predicted
+from 
+  news20b_test t JOIN news20b_adadelta_predict1 p
+    on (t.rowid = p.rowid);
+```
+
+```sql
+select count(1)/4996 from news20b_adadelta_submit1 
+where actual == predicted;
+```
+> 0.9549639711769415 (adagrad)
+
+> 0.9545636509207366 (adadelta)
+
+_Note that AdaDelta often performs better than AdaGrad._
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/370e2aa3/docs/gitbook/binaryclass/news20_dataset.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/binaryclass/news20_dataset.md b/docs/gitbook/binaryclass/news20_dataset.md
new file mode 100644
index 0000000..5ff80cd
--- /dev/null
+++ b/docs/gitbook/binaryclass/news20_dataset.md
@@ -0,0 +1,84 @@
+Get the news20b dataset.
+http://www.csie.ntu.edu.tw/~cjlin/libsvmtools/datasets/binary.html#news20.binary
+
+```sh
+cat <<EOF > conv.awk
+BEGIN{ FS=" " }
+{
+    label=\$1;
+    features=\$2;
+    for(i=3;i<=NF;i++)
+    {
+        features = features "," \$i;
+    }
+    print NR "\t" label "\t" features;
+}
+END{}
+EOF
+
+sort -R news20.binary > news20.random
+# [mac]
+# $ brew install coreutils
+# $ gsort -R news20.binary > news20.random
+head -15000 news20.random > news20.train
+tail -4996 news20.random > news20.test
+gawk -f conv.awk news20.train > news20.train.t
+gawk -f conv.awk news20.test > news20.test.t
+```
+
+## Putting data on HDFS
+```
+hadoop fs -mkdir -p /dataset/news20-binary/train
+hadoop fs -mkdir -p /dataset/news20-binary/test
+
+hadoop fs -copyFromLocal news20.train.t /dataset/news20-binary/train
+hadoop fs -copyFromLocal news20.test.t /dataset/news20-binary/test
+```
+
+## Training/test data prepareation
+```sql
+create database news20;
+use news20;
+
+delete jar /home/myui/tmp/hivemall.jar;
+add jar /home/myui/tmp/hivemall.jar;
+
+source /home/myui/tmp/define-all.hive;
+
+Create external table news20b_train (
+  rowid int,
+  label int,
+  features ARRAY<STRING>
+) ROW FORMAT DELIMITED FIELDS TERMINATED BY '\t' COLLECTION ITEMS TERMINATED BY "," STORED AS TEXTFILE LOCATION '/dataset/news20-binary/train';
+
+Create external table news20b_test (
+  rowid int, 
+  label int,
+  features ARRAY<STRING>
+) ROW FORMAT DELIMITED FIELDS TERMINATED BY '\t' COLLECTION ITEMS TERMINATED BY "," STORED AS TEXTFILE LOCATION '/dataset/news20-binary/test';
+
+set hivevar:seed=31;
+create or replace view news20b_train_x3
+as
+select 
+  * 
+from (
+select
+   amplify(3, *) as (rowid, label, features)
+from  
+   news20b_train 
+) t
+CLUSTER BY rand(${seed});
+
+create table news20b_test_exploded as
+select 
+  rowid,
+  label,
+  cast(split(feature,":")[0] as int) as feature,
+  cast(split(feature,":")[1] as float) as value
+  -- hivemall v0.3.1 or later
+  -- extract_feature(feature) as feature,
+  -- extract_weight(feature) as value
+from 
+  news20b_test LATERAL VIEW explode(addBias(features)) t AS feature;
+```
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/370e2aa3/docs/gitbook/binaryclass/news20_pa.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/binaryclass/news20_pa.md b/docs/gitbook/binaryclass/news20_pa.md
new file mode 100644
index 0000000..7763a15
--- /dev/null
+++ b/docs/gitbook/binaryclass/news20_pa.md
@@ -0,0 +1,243 @@
+## UDF preparation
+```
+delete jar /home/myui/tmp/hivemall.jar;
+add jar /home/myui/tmp/hivemall.jar;
+
+source /home/myui/tmp/define-all.hive;
+```
+
+---
+#[Perceptron]
+
+## model building
+```sql
+drop table news20b_perceptron_model1;
+create table news20b_perceptron_model1 as
+select 
+ feature,
+ voted_avg(weight) as weight
+from 
+ (select 
+     perceptron(addBias(features),label) as (feature,weight)
+  from 
+     news20b_train_x3
+ ) t 
+group by feature;
+```
+
+## prediction
+```sql
+create or replace view news20b_perceptron_predict1 
+as
+select
+  t.rowid, 
+  sum(m.weight * t.value) as total_weight,
+  case when sum(m.weight * t.value) > 0.0 then 1 else -1 end as label
+from 
+  news20b_test_exploded t LEFT OUTER JOIN
+  news20b_perceptron_model1 m ON (t.feature = m.feature)
+group by
+  t.rowid;
+```
+
+## evaluation
+```sql
+create or replace view news20b_perceptron_submit1 as
+select 
+  t.label as actual, 
+  pd.label as predicted
+from 
+  news20b_test t JOIN news20b_perceptron_predict1 pd 
+    on (t.rowid = pd.rowid);
+```
+
+```sql
+select count(1)/4996 from news20b_perceptron_submit1 
+where actual == predicted;
+```
+> 0.9459567654123299
+
+## Cleaning
+
+```sql
+drop table news20b_perceptron_model1;
+drop view news20b_perceptron_predict1;
+drop view news20b_perceptron_submit1;
+```
+
+---
+#[Passive Aggressive]
+
+## model building
+```sql
+drop table news20b_pa_model1;
+create table news20b_pa_model1 as
+select 
+ feature,
+ voted_avg(weight) as weight
+from 
+ (select 
+     train_pa(addBias(features),label) as (feature,weight)
+  from 
+     news20b_train_x3
+ ) t 
+group by feature;
+```
+
+## prediction
+```sql
+create or replace view news20b_pa_predict1 
+as
+select
+  t.rowid, 
+  sum(m.weight * t.value) as total_weight,
+  case when sum(m.weight * t.value) > 0.0 then 1 else -1 end as label
+from 
+  news20b_test_exploded t LEFT OUTER JOIN
+  news20b_pa_model1 m ON (t.feature = m.feature)
+group by
+  t.rowid;
+```
+
+## evaluation
+```
+create or replace view news20b_pa_submit1 as
+select 
+  t.label as actual, 
+  pd.label as predicted
+from 
+  news20b_test t JOIN news20b_pa_predict1 pd 
+    on (t.rowid = pd.rowid);
+```
+
+```sql
+select count(1)/4996 from news20b_pa_submit1 
+where actual == predicted;
+```
+> 0.9603682946357086
+
+## Cleaning
+
+```sql
+drop table news20b_pa_model1;
+drop view news20b_pa_predict1;
+drop view news20b_pa_submit1;
+```
+
+---
+#[Passive Aggressive (PA1)]
+
+## model building
+```sql
+drop table news20b_pa1_model1;
+create table news20b_pa1_model1 as
+select 
+ feature,
+ voted_avg(weight) as weight
+from 
+ (select 
+     train_pa1(addBias(features),label) as (feature,weight)
+  from 
+     news20b_train_x3
+ ) t 
+group by feature;
+```
+
+## prediction
+```sql
+create or replace view news20b_pa1_predict1 
+as
+select
+  t.rowid, 
+  sum(m.weight * t.value) as total_weight,
+  case when sum(m.weight * t.value) > 0.0 then 1 else -1 end as label
+from 
+  news20b_test_exploded t LEFT OUTER JOIN
+  news20b_pa1_model1 m ON (t.feature = m.feature)
+group by
+  t.rowid;
+```
+
+## evaluation
+```sql
+create or replace view news20b_pa1_submit1 as
+select 
+  t.label as actual, 
+  pd.label as predicted
+from 
+  news20b_test t JOIN news20b_pa1_predict1 pd 
+    on (t.rowid = pd.rowid);
+```
+
+```sql
+select count(1)/4996 from news20b_pa1_submit1 
+where actual == predicted;
+```
+> 0.9601681345076061
+
+## Cleaning
+
+```sql
+drop table news20b_pa1_model1;
+drop view news20b_pa1_predict1;
+drop view news20b_pa1_submit1;
+```
+
+---
+#[Passive Aggressive (PA2)]
+
+## model building
+```sql
+drop table news20b_pa2_model1;
+create table news20b_pa2_model1 as
+select 
+ feature,
+ voted_avg(weight) as weight
+from 
+ (select 
+     train_pa2(addBias(features),label) as (feature,weight)
+  from 
+     news20b_train_x3
+ ) t 
+group by feature;
+```
+
+## prediction
+```sql
+create or replace view news20b_pa2_predict1 
+as
+select
+  t.rowid, 
+  sum(m.weight * t.value) as total_weight,
+  case when sum(m.weight * t.value) > 0.0 then 1 else -1 end as label
+from 
+  news20b_test_exploded t LEFT OUTER JOIN
+  news20b_pa2_model1 m ON (t.feature = m.feature)
+group by
+  t.rowid;
+```
+
+## evaluation
+```sql
+create or replace view news20b_pa2_submit1 as
+select 
+  t.label as actual, 
+  pd.label as predicted
+from 
+  news20b_test t JOIN news20b_pa2_predict1 pd 
+    on (t.rowid = pd.rowid);
+```
+
+```sql
+select count(1)/4996 from news20b_pa2_submit1 
+where actual == predicted;
+```
+> 0.9597678142514011
+
+## Cleaning
+
+```sql
+drop table news20b_pa2_model1;
+drop view news20b_pa2_predict1;
+drop view news20b_pa2_submit1;
+```
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/370e2aa3/docs/gitbook/binaryclass/news20_scw.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/binaryclass/news20_scw.md b/docs/gitbook/binaryclass/news20_scw.md
new file mode 100644
index 0000000..0b27dab
--- /dev/null
+++ b/docs/gitbook/binaryclass/news20_scw.md
@@ -0,0 +1,272 @@
+## UDF preparation
+```
+use news20;
+
+delete jar /home/myui/tmp/hivemall.jar;
+add jar /home/myui/tmp/hivemall.jar;
+source /home/myui/tmp/define-all.hive;
+```
+
+---
+# Confidece Weighted (CW)
+
+## training
+```sql
+drop table news20b_cw_model1;
+create table news20b_cw_model1 as
+select 
+ feature,
+ -- voted_avg(weight) as weight -- [hivemall v0.1]
+ argmin_kld(weight, covar) as weight -- [hivemall v0.2 or later]
+from 
+ (select 
+     -- train_cw(addBias(features), label) as (feature, weight) -- [hivemall v0.1]
+     train_cw(addBias(features), label) as (feature, weight, covar) -- [hivemall v0.2 or later]
+  from 
+     news20b_train_x3
+ ) t 
+group by feature;
+```
+
+## prediction
+```sql
+create or replace view news20b_cw_predict1 
+as
+select
+  t.rowid, 
+  sum(m.weight * t.value) as total_weight,
+  case when sum(m.weight * t.value) > 0.0 then 1 else -1 end as label
+from 
+  news20b_test_exploded t LEFT OUTER JOIN
+  news20b_cw_model1 m ON (t.feature = m.feature)
+group by
+  t.rowid;
+```
+
+## evaluation
+```sql
+create or replace view news20b_cw_submit1 
+as
+select 
+  t.rowid,
+  t.label as actual, 
+  pd.label as predicted
+from 
+  news20b_test t JOIN news20b_cw_predict1 pd 
+    on (t.rowid = pd.rowid);
+```
+
+```sql
+select count(1)/4996 from news20b_cw_submit1 
+where actual = predicted;
+```
+> 0.9655724579663731
+
+## Cleaning
+
+```sql
+drop table news20b_cw_model1;
+drop view news20b_cw_predict1;
+drop view news20b_cw_submit1;
+```
+
+---
+# Adaptive Regularization of Weight Vectors (AROW)
+
+## training
+```sql
+drop table news20b_arow_model1;
+create table news20b_arow_model1 as
+select 
+ feature,
+ -- voted_avg(weight) as weight -- [hivemall v0.1]
+ argmin_kld(weight, covar) as weight -- [hivemall v0.2 or later]
+from 
+ (select 
+     -- train_arow(addBias(features),label) as (feature,weight) -- [hivemall v0.1]
+     train_arow(addBias(features),label) as (feature,weight,covar) -- [hivemall v0.2 or later]
+  from 
+     news20b_train_x3
+ ) t 
+group by feature;
+```
+
+## prediction
+```sql
+create or replace view news20b_arow_predict1 
+as
+select
+  t.rowid, 
+  sum(m.weight * t.value) as total_weight,
+  case when sum(m.weight * t.value) > 0.0 then 1 else -1 end as label
+from 
+  news20b_test_exploded t LEFT OUTER JOIN
+  news20b_arow_model1 m ON (t.feature = m.feature)
+group by
+  t.rowid;
+```
+
+## evaluation
+```sql
+create or replace view news20b_arow_submit1 as
+select 
+  t.rowid, 
+  t.label as actual, 
+  pd.label as predicted
+from 
+  news20b_test t JOIN news20b_arow_predict1 pd 
+    on (t.rowid = pd.rowid);
+```
+
+```sql
+select count(1)/4996 from news20b_arow_submit1 
+where actual = predicted;
+```
+> 0.9659727782225781
+
+## Cleaning
+
+```sql
+drop table news20b_arow_model1;
+drop view news20b_arow_predict1;
+drop view news20b_arow_submit1;
+```
+
+---
+# Soft Confidence-Weighted (SCW1)
+
+## training
+```sql
+drop table news20b_scw_model1;
+create table news20b_scw_model1 as
+select 
+ feature,
+ -- voted_avg(weight) as weight -- [hivemall v0.1]
+ argmin_kld(weight, covar) as weight -- [hivemall v0.2 or later]
+from 
+ (select 
+     -- train_scw(addBias(features),label) as (feature,weight) -- [hivemall v0.1]
+     train_scw(addBias(features),label) as (feature,weight,covar) -- [hivemall v0.2 or later]
+  from 
+     news20b_train_x3
+ ) t 
+group by feature;
+```
+
+## prediction
+```sql
+create or replace view news20b_scw_predict1 
+as
+select
+  t.rowid, 
+  sum(m.weight * t.value) as total_weight,
+  case when sum(m.weight * t.value) > 0.0 then 1 else -1 end as label
+from 
+  news20b_test_exploded t LEFT OUTER JOIN
+  news20b_scw_model1 m ON (t.feature = m.feature)
+group by
+  t.rowid;
+```
+
+## evaluation
+```sql
+create or replace view news20b_scw_submit1 as
+select 
+  t.rowid, 
+  t.label as actual, 
+  pd.label as predicted
+from 
+  news20b_test t JOIN news20b_scw_predict1 pd 
+    on (t.rowid = pd.rowid);
+```
+
+```sql
+select count(1)/4996 from news20b_scw_submit1 
+where actual = predicted;
+```
+> 0.9661729383506805
+
+## Cleaning
+
+```sql
+drop table news20b_scw_model1;
+drop view news20b_scw_predict1;
+drop view news20b_scw_submit1;
+```
+
+---
+# Soft Confidence-Weighted (SCW2)
+
+## training
+```sql
+drop table news20b_scw2_model1;
+create table news20b_scw2_model1 as
+select 
+ feature,
+ -- voted_avg(weight) as weight -- [hivemall v0.1]
+ argmin_kld(weight, covar) as weight -- [hivemall v0.2 or later]
+from 
+ (select 
+     -- train_scw2(addBias(features),label) as (feature,weight)    -- [hivemall v0.1]
+     train_scw2(addBias(features),label) as (feature,weight,covar) -- [hivemall v0.2 or later]
+  from 
+     news20b_train_x3
+ ) t 
+group by feature;
+```
+
+## prediction
+```sql
+create or replace view news20b_scw2_predict1 
+as
+select
+  t.rowid, 
+  sum(m.weight * t.value) as total_weight,
+  case when sum(m.weight * t.value) > 0.0 then 1 else -1 end as label
+from 
+  news20b_test_exploded t LEFT OUTER JOIN
+  news20b_scw2_model1 m ON (t.feature = m.feature)
+group by
+  t.rowid;
+```
+
+## evaluation
+```sql
+create or replace view news20b_scw2_submit1 as
+select 
+  t.rowid, 
+  t.label as actual, 
+  pd.label as predicted
+from 
+  news20b_test t JOIN news20b_scw2_predict1 pd 
+    on (t.rowid = pd.rowid);
+```
+
+```sql
+select count(1)/4996 from news20b_scw2_submit1 
+where actual = predicted;
+```
+> 0.9579663730984788
+
+## Cleaning
+
+```sql
+drop table news20b_scw2_model1;
+drop view news20b_scw2_predict1;
+drop view news20b_scw2_submit1;
+```
+
+--
+
+| Algorithm | Accuracy |
+|:-----------|------------:|
+| Perceptron | 0.9459567654123299 |
+| SCW2 | 0.9579663730984788 |
+| PA2 | 0.9597678142514011 |
+| PA1 | 0.9601681345076061 |
+| PA | 0.9603682946357086 |
+| CW | 0.9655724579663731 |
+| AROW | 0.9659727782225781 |
+| SCW1 | 0.9661729383506805 |
+
+My recommendation is AROW for classification.
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/370e2aa3/docs/gitbook/binaryclass/webspam.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/binaryclass/webspam.md b/docs/gitbook/binaryclass/webspam.md
new file mode 100644
index 0000000..e69de29

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/370e2aa3/docs/gitbook/binaryclass/webspam_dataset.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/binaryclass/webspam_dataset.md b/docs/gitbook/binaryclass/webspam_dataset.md
new file mode 100644
index 0000000..4686865
--- /dev/null
+++ b/docs/gitbook/binaryclass/webspam_dataset.md
@@ -0,0 +1,76 @@
+Get the dataset from 
+http://www.csie.ntu.edu.tw/~cjlin/libsvmtools/datasets/binary.html#webspam
+
+# Putting data on HDFS
+```sql
+hadoop fs -mkdir -p /dataset/webspam/raw
+
+awk -f conv.awk webspam_wc_normalized_trigram.svm | \
+hadoop fs -put - /dataset/webspam/raw/
+```
+
+# Table preparation
+```sql
+create database webspam;
+use webspam;
+
+delete jar ./tmp/hivemall.jar;
+add jar ./tmp/hivemall.jar;
+source ./tmp/define-all.hive;
+
+create external table webspam_raw (
+  rowid int,
+  label int,
+  features ARRAY<STRING>
+) ROW FORMAT 
+DELIMITED FIELDS TERMINATED BY '\t' 
+COLLECTION ITEMS TERMINATED BY "," 
+STORED AS TEXTFILE LOCATION '/dataset/webspam/raw';
+
+set hive.sample.seednumber=43;
+create table webspam_test
+as
+select * from webspam_raw TABLESAMPLE(1000 ROWS) s
+CLUSTER BY rand(43)
+limit 70000;
+```
+
+# Make auxiliary tables
+```sql
+create table webspam_train_orcfile (
+ rowid int,
+ label int,
+ features array<string>
+) STORED AS orc tblproperties ("orc.compress"="SNAPPY");
+
+-- SET mapred.reduce.tasks=128;
+INSERT OVERWRITE TABLE webspam_train_orcfile
+select
+  s.rowid, 
+  label,
+  addBias(features) as features
+from webspam_raw s
+where not exists (select rowid from webspam_test t where s.rowid = t.rowid)
+CLUSTER BY rand(43);
+-- SET mapred.reduce.tasks=-1;
+
+set hivevar:xtimes=3;
+set hivevar:shufflebuffersize=100;
+set hivemall.amplify.seed=32;
+create or replace view webspam_train_x3
+as
+select
+   rand_amplify(${xtimes}, ${shufflebuffersize}, *) as (rowid, label, features)
+from  
+   webspam_train_orcfile;
+
+create table webspam_test_exploded as
+select 
+  rowid,
+  label,
+  split(feature,":")[0] as feature,
+  cast(split(feature,":")[1] as float) as value
+from 
+  webspam_test LATERAL VIEW explode(addBias(features)) t AS feature;
+```
+*Caution:* For this dataset, use small *shufflebuffersize* because each training example has lots of features though (xtimes * shufflebuffersize * N) training examples are cached in memory.
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/370e2aa3/docs/gitbook/binaryclass/webspam_scw.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/binaryclass/webspam_scw.md b/docs/gitbook/binaryclass/webspam_scw.md
new file mode 100644
index 0000000..635b32d
--- /dev/null
+++ b/docs/gitbook/binaryclass/webspam_scw.md
@@ -0,0 +1,136 @@
+# Preparation
+
+```
+use webspam;
+
+delete jar ./tmp/hivemall.jar;
+add jar ./tmp/hivemall.jar;
+source ./tmp/define-all.hive;
+```
+
+# PA1
+
+```sql
+drop table webspam_pa1_model1;
+create table webspam_pa1_model1 as
+select 
+ feature,
+ cast(voted_avg(weight) as float) as weight
+from 
+ (select 
+     train_pa1(features,label) as (feature,weight) -- sparse model
+     -- train_pa1(features,label,"-dense -dims 33554432") as (feature,weight)
+  from 
+     webspam_train_x3
+ ) t 
+group by feature;
+
+create or replace view webspam_pa1_predict1 
+as
+select
+  t.rowid, 
+  sum(m.weight * t.value) as total_weight,
+  case when sum(m.weight * t.value) > 0.0 then 1 else -1 end as label
+from 
+  webspam_test_exploded t LEFT OUTER JOIN
+  webspam_pa1_model1 m ON (t.feature = m.feature)
+group by
+  t.rowid;
+
+create or replace view webspam_pa1_submit1 as
+select 
+  t.rowid, 
+  t.label as actual, 
+  pd.label as predicted
+from 
+  webspam_test t JOIN webspam_pa1_predict1 pd 
+    on (t.rowid = pd.rowid);
+
+select count(1)/70000 from webspam_pa1_submit1 
+where actual = predicted;
+```
+> Prediction accuracy: 0.9628428571428571
+
+# AROW
+
+```sql
+drop table webspam_arow_model1;
+create table webspam_arow_model1 as
+select 
+ feature,
+ argmin_kld(weight,covar)as weight
+from 
+ (select 
+     train_arow(features,label) as (feature,weight,covar)
+  from 
+     webspam_train_x3
+ ) t 
+group by feature;
+
+create or replace view webspam_arow_predict1 
+as
+select
+  t.rowid, 
+  sum(m.weight * t.value) as total_weight,
+  case when sum(m.weight * t.value) > 0.0 then 1 else -1 end as label
+from 
+  webspam_test_exploded t LEFT OUTER JOIN
+  webspam_arow_model1 m ON (t.feature = m.feature)
+group by
+  t.rowid;
+
+create or replace view webspam_arow_submit1 as
+select 
+  t.rowid, 
+  t.label as actual, 
+  pd.label as predicted
+from 
+  webspam_test t JOIN webspam_arow_predict1 pd 
+    on (t.rowid = pd.rowid);
+
+select count(1)/70000 from webspam_arow_submit1 
+where actual = predicted;
+```
+> Prediction accuracy: 0.9747428571428571
+
+# SCW1
+
+```sql
+drop table webspam_scw_model1;
+create table webspam_scw_model1 as
+select 
+ feature,
+ argmin_kld(weight,covar)as weight
+from 
+ (select 
+     train_scw(features,label) as (feature,weight,covar)
+  from 
+     webspam_train_x3
+ ) t 
+group by feature;
+
+create or replace view webspam_scw_predict1 
+as
+select
+  t.rowid, 
+  sum(m.weight * t.value) as total_weight,
+  case when sum(m.weight * t.value) > 0.0 then 1 else -1 end as label
+from 
+  webspam_test_exploded t LEFT OUTER JOIN
+  webspam_scw_model1 m ON (t.feature = m.feature)
+group by
+  t.rowid;
+
+create or replace view webspam_scw_submit1 as
+select 
+  t.rowid, 
+  t.label as actual, 
+  pd.label as predicted
+from 
+  webspam_test t JOIN webspam_scw_predict1 pd 
+    on (t.rowid = pd.rowid);
+
+select count(1)/70000 from webspam_scw_submit1 
+where actual = predicted;
+```
+> Prediction accuracy: 0.9778714285714286
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/370e2aa3/docs/gitbook/book.json
----------------------------------------------------------------------
diff --git a/docs/gitbook/book.json b/docs/gitbook/book.json
new file mode 100644
index 0000000..2f70ed9
--- /dev/null
+++ b/docs/gitbook/book.json
@@ -0,0 +1,64 @@
+{
+    "gitbook": "3.x.x",
+    "title": "Hivemall User Manual",
+    "description": "User Manual for Apache Hivemall",       
+    "plugins": [
+        "theme-api",
+        "edit-link",
+        "github",
+        "splitter",
+        "sitemap",
+        "etoc",
+        "callouts",
+        "toggle-chapters",
+        "anchorjs",
+        "codeblock-filename",
+        "expandable-chapters",
+        "multipart",
+        "codeblock-filename",
+        "katex",
+        "emphasize"
+    ],
+    "pluginsConfig": {
+        "theme-default": {
+            "showLevel": true
+        },
+        "theme-api": {
+            "theme": "dark",
+            "split": false
+        },
+        "edit-link": {
+            "base": "https://github.com/apache/incubator-hivemall/docs/gitbook",
+            "label": "Edit"
+        },
+        "github": {
+            "url": "https://github.com/apache/incubator-hivemall/"
+        },
+        "sitemap": {
+            "hostname": "http://hivemall.incubator.apache.org/"
+        },
+        "etoc": {
+          "mindepth": 1,
+          "maxdepth": 3,
+          "notoc": true
+        },
+        "downloadpdf": {
+            "base": "https://github.com/apache/incubator-hivemall/docs/gitbook",
+            "label": "PDF",
+            "multilingual": false
+        },
+        "fontsettings": {
+            "theme": "white",
+            "font": "sans",
+            "size": 2
+        },
+        "anchorjs": {
+            "selector": "h1,h2,h3,*:not(.callout) > h4,h5"
+        }
+    },
+    "links": {
+      "sidebar": {
+        "<i class=\"fa fa-home\"></i> Home": "http://hivemall.incubator.apache.org/"
+      }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/370e2aa3/docs/gitbook/eval/datagen.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/eval/datagen.md b/docs/gitbook/eval/datagen.md
new file mode 100644
index 0000000..e69de29

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/370e2aa3/docs/gitbook/eval/lr_datagen.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/eval/lr_datagen.md b/docs/gitbook/eval/lr_datagen.md
new file mode 100644
index 0000000..55cb360
--- /dev/null
+++ b/docs/gitbook/eval/lr_datagen.md
@@ -0,0 +1,92 @@
+_Note this feature is supported on hivemall v0.2-alpha3 or later._
+
+# create a dual table
+
+Create a [dual table](http://en.wikipedia.org/wiki/DUAL_table) as follows:
+```sql
+CREATE TABLE dual (
+  dummy int
+);
+INSERT INTO TABLE dual SELECT count(*)+1 FROM dual;
+```
+
+# Sparse dataset generation by a single task
+```sql
+create table regression_data1
+as
+select lr_datagen("-n_examples 10k -n_features 10 -seed 100") as (label,features)
+from dual;
+```
+Find the details of the option in [LogisticRegressionDataGeneratorUDTF.java](https://github.com/myui/hivemall/blob/master/core/src/main/java/hivemall/dataset/LogisticRegressionDataGeneratorUDTF.java#L69).
+
+You can generate a sparse dataset as well as a dense dataset. By the default, a sparse dataset is generated.
+```sql
+hive> desc regression_data1;
+OK
+label                   float                   None
+features                array<string>           None
+
+hive> select * from regression_data1 limit 2;
+OK
+0.7220096       ["140:2.8347101","165:3.0056276","179:4.030076","112:3.3919246","99:3.98914","16:3.5653272","128:3.046535","124:2.7708225","78:2.4960368","6:1.7866131"]
+0.7346627       ["139:1.9607254","110:2.958568","186:3.2524762","31:3.9243593","167:0.72854257","26:1.8355447","117:2.7663715","3:2.1551287","179:3.1099443","19:3.6411424"]
+Time taken: 0.046 seconds, Fetched: 2 row(s)
+```
+
+# Classification dataset generation
+You can use "-cl" option to generation 0/1 label.
+```sql
+select lr_datagen("-cl") as (label,features)
+from dual 
+limit 5;
+OK
+1.0     ["84:3.4227803","80:3.8875976","58:3.2909582","123:3.1056073","194:3.3360343","199:2.20207","75:3.5469763","74:3.3869767","126:0.9969454","93:2.5352612"]
+0.0     ["84:-0.5568947","10:0.621897","6:-0.13126314","190:0.18610542","131:1.7232913","24:-2.7551131","113:-0.9842969","177:0.062993184","176:-0.19020283","21:-0.54811275"]
+1.0     ["73:3.4391513","198:4.42387","164:4.248151","66:3.5224934","84:1.9026604","76:0.79803777","18:2.2168183","163:2.248695","119:1.5906067","72:2.0267224"]
+1.0     ["34:2.9269936","35:0.37033868","39:3.771989","47:2.2087111","28:2.9445739","55:4.134555","14:2.4297745","164:3.0913055","52:2.0519433","128:2.9108515"]
+1.0     ["98:4.2451696","4:3.486905","133:2.4589922","26:2.7301126","103:2.6827147","2:3.6198254","34:3.7042716","47:2.5515237","68:2.4294896","197:4.4958663"]
+```
+
+# Dense dataset generation
+```sql
+create table regression_data_dense
+as
+select lr_datagen("-dense -n_examples 9999 -n_features 100 -n_dims 100") as (label,features)
+from dual;
+
+hive> desc regression_data_dense;
+OK
+label                   float                   None
+features                array<float>            None
+
+hive> select * from regression_data_dense limit 1;
+OK
+0.7274741       [4.061373,3.9373128,3.5195694,3.3604698,3.7698417,4.2518,3.8796813,1.6020582,4.937072,1.5513933,3.0289552,2.6674519,3.432688,2.980945,1.8897587,2.9770515,3.3435504,1.7867403,3.4057906,1.2151588,5.0587463,2.1410913,2.8097973,2.4518871,3.175268,3.3347685,3.728993,3.1443396,3.5506077,3.6357877,4.248151,3.5224934,3.2423255,2.5188355,1.8626233,2.8432152,2.2762651,4.57472,2.2168183,2.248695,3.3636255,2.8359523,2.0327945,1.5917025,2.9269936,0.37033868,2.6151125,4.545956,2.0863252,3.7857852,2.9445739,4.134555,3.0660007,3.4279037,2.0519433,2.9108515,3.5171766,3.4708095,3.161707,2.39229,2.4589922,2.7301126,3.5303073,2.7398396,3.7042716,2.5515237,3.0943663,0.41565156,4.672767,3.1461313,3.0443575,3.4023938,2.2205734,1.8950733,2.1664586,4.8654623,2.787029,4.0460386,2.4455893,3.464298,1.062505,3.0513604,4.382525,2.771433,3.2828436,3.803544,2.178681,4.2466116,3.5440445,3.1546876,3.4248536,0.9067459,3.0134914,1.9528451,1.7175893,2.7029774,2.5759792,3.643847,3.0799,3.735559]
+Time taken: 0.044 seconds, Fetched: 1 row(s)
+```
+
+# Parallel and scalable data generation using multiple reducers (RECOMMENDED)
+Dataset generation using (at max) 10 reducers.
+
+```sql
+set hivevar:n_parallel_datagen=10;
+
+create or replace view seq10 
+as
+select * from (
+  select generate_series(1,${n_parallel_datagen})
+  from dual 
+) t
+DISTRIBUTE BY value;
+
+set mapred.reduce.tasks=${n_parallel_datagen};
+create table lrdata1k
+as
+select lr_datagen("-n_examples 100")
+from seq10;
+set mapred.reduce.tasks=-1; -- reset to the default setting
+
+hive> select count(1) from lrdata1k;
+OK
+1000
+```
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/370e2aa3/docs/gitbook/eval/stat_eval.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/eval/stat_eval.md b/docs/gitbook/eval/stat_eval.md
new file mode 100644
index 0000000..7f1688b
--- /dev/null
+++ b/docs/gitbook/eval/stat_eval.md
@@ -0,0 +1,56 @@
+Using the [E2006 tfidf regression example](https://github.com/myui/hivemall/wiki/E2006-tfidf-regression-evaluation-(PA,-AROW)), we explain how to evaluate the prediction model on Hive.
+
+# Scoring by evaluation metrics
+
+```sql
+select avg(actual), avg(predicted) from e2006tfidf_pa2a_submit;
+```
+> -3.8200363760415414     -3.9124877451612488
+
+```sql
+set hivevar:mean_actual=-3.8200363760415414;
+
+select 
+-- Root Mean Squared Error
+   rmse(predicted, actual) as RMSE, 
+   -- sqrt(sum(pow(predicted - actual,2.0))/count(1)) as RMSE,
+-- Mean Squared Error
+   mse(predicted, actual) as MSE, 
+   -- sum(pow(predicted - actual,2.0))/count(1) as MSE,
+-- Mean Absolute Error
+   mae(predicted, actual) as MAE, 
+   -- sum(abs(predicted - actual))/count(1) as MAE,
+-- coefficient of determination (R^2)
+   -- 1 - sum(pow(actual - predicted,2.0)) / sum(pow(actual - ${mean_actual},2.0)) as R2
+   r2(actual, predicted) as R2 -- supported since Hivemall v0.4.1-alpha.5
+from 
+   e2006tfidf_pa2a_submit;
+```
+> 0.38538660838804495     0.14852283792484033     0.2466732002711477      0.48623913673053565
+
+# Logarithmic Loss
+
+[Logarithmic Loss](https://www.kaggle.com/wiki/LogarithmicLoss) can be computed as follows:
+
+```sql
+WITH t as ( 
+  select 
+    0 as actual,
+    0.01 as predicted
+  union all
+  select 
+    1 as actual,
+    0.02 as predicted
+)
+select 
+   -SUM(actual*LN(predicted)+(1-actual)*LN(1-predicted))/count(1) as logloss1,
+  logloss(predicted, actual) as logloss2 -- supported since Hivemall v0.4.2-rc.1
+from 
+from t;
+```
+> 1.9610366706408238	1.9610366706408238
+
+--
+**References**
+* R2 http://en.wikipedia.org/wiki/Coefficient_of_determination
+* Evaluation Metrics https://www.kaggle.com/wiki/Metrics
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/370e2aa3/docs/gitbook/ft_engineering/ft_trans.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/ft_engineering/ft_trans.md b/docs/gitbook/ft_engineering/ft_trans.md
new file mode 100644
index 0000000..e69de29

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/370e2aa3/docs/gitbook/ft_engineering/hashing.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/ft_engineering/hashing.md b/docs/gitbook/ft_engineering/hashing.md
new file mode 100644
index 0000000..09fa1ff
--- /dev/null
+++ b/docs/gitbook/ft_engineering/hashing.md
@@ -0,0 +1,103 @@
+Hivemall supports [Feature Hashing](https://github.com/myui/hivemall/wiki/Feature-hashing) (a.k.a. hashing trick) through `feature_hashing` and `mhash` functions. 
+Find the differences in the following examples.
+
+_Note: `feature_hashing` UDF is supported since Hivemall `v0.4.2-rc.1`._
+
+## `feature_hashing` function
+
+`feature_hashing` applies MurmuerHash3 hashing to features. 
+
+```sql
+select feature_hashing('aaa');
+> 4063537
+
+select feature_hashing('aaa','-features 3');
+> 2
+
+select feature_hashing(array('aaa','bbb'));
+> ["4063537","8459207"]
+
+select feature_hashing(array('aaa','bbb'),'-features 10');
+> ["7","1"]
+
+select feature_hashing(array('aaa:1.0','aaa','bbb:2.0'));
+> ["4063537:1.0","4063537","8459207:2.0"]
+
+select feature_hashing(array(1,2,3));
+> ["11293631","3322224","4331412"]
+
+select feature_hashing(array('1','2','3'));
+> ["11293631","3322224","4331412"]
+
+select feature_hashing(array('1:0.1','2:0.2','3:0.3'));
+> ["11293631:0.1","3322224:0.2","4331412:0.3"]
+
+select feature_hashing(features), features from training_fm limit 2;
+
+> ["1803454","6630176"]   ["userid#5689","movieid#3072"]
+> ["1828616","6238429"]   ["userid#4505","movieid#2331"]
+
+select feature_hashing(array("userid#4505:3.3","movieid#2331:4.999", "movieid#2331"));
+
+> ["1828616:3.3","6238429:4.999","6238429"]
+```
+
+_Note: The hash value is starting from 1 and 0 is system reserved for a bias clause. The default number of features are 16777217 (2^24). You can control the number of features by `-num_features` (or `-features`) option._
+
+```sql
+select feature_hashing(null,'-help');
+
+usage: feature_hashing(array<string> features [, const string options]) -
+       returns a hashed feature vector in array<string> [-features <arg>]
+       [-help]
+ -features,--num_features <arg>   The number of features [default:
+                                  16777217 (2^24)]
+ -help                            Show function help
+```
+
+## `mhash` function
+
+```sql
+describe function extended mhash;
+> mhash(string word) returns a murmurhash3 INT value starting from 1
+```
+
+```sql
+
+select mhash('aaa');
+> 4063537
+```
+
+_Note: The default number of features are `16777216 (2^24)`._
+```sql
+set hivevar:num_features=16777216;
+
+select mhash('aaa',${num_features});
+>4063537
+```
+
+_Note: `mhash` returns a `+1'd` murmerhash3 value starting from 1. Never returns 0 (It's a system reserved number)._
+```sql
+set hivevar:num_features=1;
+
+select mhash('aaa',${num_features});
+> 1
+```
+
+_Note: `mhash` does not considers feature values._
+```sql
+select mhash('aaa:2.0');
+> 2746618
+```
+
+_Note: `mhash` always returns a scalar INT value._
+```sql
+select mhash(array('aaa','bbb'));
+> 9566153
+```
+
+_Note: `mhash` value of an array is element order-sentitive._
+```sql
+select mhash(array('bbb','aaa'));
+> 3874068
+```
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/370e2aa3/docs/gitbook/ft_engineering/quantify.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/ft_engineering/quantify.md b/docs/gitbook/ft_engineering/quantify.md
new file mode 100644
index 0000000..1d6a223
--- /dev/null
+++ b/docs/gitbook/ft_engineering/quantify.md
@@ -0,0 +1,164 @@
+`quantified_features` is useful for transforming values of non-number columns to indexed numbers.
+
+*Note: The feature is supported Hivemall v0.4 or later.*
+
+```sql
+desc train;
+
+id                      int                                         
+age                     int                                         
+job                     string                                      
+marital                 string                                      
+education               string                                      
+default                 string                                      
+balance                 int                                         
+housing                 string                                      
+loan                    string                                      
+contact                 string                                      
+day                     int                                         
+month                   string                                      
+duration                int                                         
+campaign                int                                         
+pdays                   int                                         
+previous                int                                         
+poutcome                string                                      
+y                       int
+```
+
+```sql
+select * from train limit 10;
+
+1       39      blue-collar     married secondary       no      1756    yes     no      cellular        3       apr     939     1       -1      0       unknown 1
+2       51      entrepreneur    married primary no      1443    no      no      cellular        18      feb     172     10      -1      0       unknown 1
+3       36      management      single  tertiary        no      436     no      no      cellular        13      apr     567     1       595     2       failure 1
+4       63      retired married secondary       no      474     no      no      cellular        25      jan     423     1       -1      0       unknown 1
+5       31      management      single  tertiary        no      354     no      no      cellular        30      apr     502     1       9       2       success 1
+6       29      blue-collar     single  secondary       no      260     yes     no      unknown 2       jun     707     14      -1      0       unknown 1
+7       37      services        married secondary       no      52      yes     no      cellular        6       sep     908     1       185     9       success 1
+8       32      technician      single  secondary       no      230     yes     no      cellular        18      may     442     1       266     8       failure 1
+9       31      admin.  single  secondary       no      0       yes     no      cellular        7       may     895     2       295     2       failure 1
+10      32      self-employed   single  tertiary        no      1815    no      no      telephone       10      aug     235     1       102     2       failure 1
+```
+
+```sql
+set hivevar:output_row=true;
+
+select quantify(${output_row}, *) 
+from (
+  select * from train
+  order by id asc -- force quantify() runs on a single reducer
+) t
+limit 10;
+
+1       39      0       0       0       0       1756    0       0       0       3       0       939     1       -1      0       0       1
+2       51      1       0       1       0       1443    1       0       0       18      1       172     10      -1      0       0       1
+3       36      2       1       2       0       436     1       0       0       13      0       567     1       595     2       1       1
+4       63      3       0       0       0       474     1       0       0       25      2       423     1       -1      0       0       1
+5       31      2       1       2       0       354     1       0       0       30      0       502     1       9       2       2       1
+6       29      0       1       0       0       260     0       0       1       2       3       707     14      -1      0       0       1
+7       37      4       0       0       0       52      0       0       0       6       4       908     1       185     9       2       1
+8       32      5       1       0       0       230     0       0       0       18      5       442     1       266     8       1       1
+9       31      6       1       0       0       0       0       0       0       7       5       895     2       295     2       1       1
+10      32      7       1       2       0       1815    1       0       2       10      6       235     1       102     2       1       1
+```
+
+```sql
+select 
+  quantify(
+    ${output_row}, id, age, job, marital, education, default, balance, housing, loan, contact, day, month, duration, campaign, cast(pdays as string), previous, poutcome, y
+  ) as (id, age, job, marital, education, default, balance, housing, loan, contact, day, month, duration, campaign, pdays, previous, poutcome, y)
+from (
+  select * from train
+  order by id asc
+) t
+limit 10;
+
+1       39      0       0       0       0       1756    0       0       0       3       0       939     1       0       0       0       1
+2       51      1       0       1       0       1443    1       0       0       18      1       172     10      0       0       0       1
+3       36      2       1       2       0       436     1       0       0       13      0       567     1       1       2       1       1
+4       63      3       0       0       0       474     1       0       0       25      2       423     1       0       0       0       1
+5       31      2       1       2       0       354     1       0       0       30      0       502     1       2       2       2       1
+6       29      0       1       0       0       260     0       0       1       2       3       707     14      0       0       0       1
+7       37      4       0       0       0       52      0       0       0       6       4       908     1       3       9       2       1
+8       32      5       1       0       0       230     0       0       0       18      5       442     1       4       8       1       1
+9       31      6       1       0       0       0       0       0       0       7       5       895     2       5       2       1       1
+10      32      7       1       2       0       1815    1       0       2       10      6       235     1       6       2       1       1
+```
+
+```sql
+select 
+  quantified_features(
+    ${output_row}, id, age, job, marital, education, default, balance, housing, loan, contact, day, month, duration, campaign, cast(pdays as string), previous, poutcome, y
+  ) as features
+from (
+  select * from train
+  order by id asc
+) t
+limit 10;
+
+[1.0,39.0,0.0,0.0,0.0,0.0,1756.0,0.0,0.0,0.0,3.0,0.0,939.0,1.0,0.0,0.0,0.0,1.0]
+[2.0,51.0,1.0,0.0,1.0,0.0,1443.0,1.0,0.0,0.0,18.0,1.0,172.0,10.0,0.0,0.0,0.0,1.0]
+[3.0,36.0,2.0,1.0,2.0,0.0,436.0,1.0,0.0,0.0,13.0,0.0,567.0,1.0,1.0,2.0,1.0,1.0]
+[4.0,63.0,3.0,0.0,0.0,0.0,474.0,1.0,0.0,0.0,25.0,2.0,423.0,1.0,0.0,0.0,0.0,1.0]
+[5.0,31.0,2.0,1.0,2.0,0.0,354.0,1.0,0.0,0.0,30.0,0.0,502.0,1.0,2.0,2.0,2.0,1.0]
+[6.0,29.0,0.0,1.0,0.0,0.0,260.0,0.0,0.0,1.0,2.0,3.0,707.0,14.0,0.0,0.0,0.0,1.0]
+[7.0,37.0,4.0,0.0,0.0,0.0,52.0,0.0,0.0,0.0,6.0,4.0,908.0,1.0,3.0,9.0,2.0,1.0]
+[8.0,32.0,5.0,1.0,0.0,0.0,230.0,0.0,0.0,0.0,18.0,5.0,442.0,1.0,4.0,8.0,1.0,1.0]
+[9.0,31.0,6.0,1.0,0.0,0.0,0.0,0.0,0.0,0.0,7.0,5.0,895.0,2.0,5.0,2.0,1.0,1.0]
+[10.0,32.0,7.0,1.0,2.0,0.0,1815.0,1.0,0.0,2.0,10.0,6.0,235.0,1.0,6.0,2.0,1.0,1.0]
+```
+
+## Quantify test dataset 
+
+```sql
+select * from test limit 10;
+
+1       30      management      single  tertiary        no      1028    no      no      cellular        4       feb     1294    2       -1      0       unknown
+2       39      self-employed   single  tertiary        no      426     no      no      unknown 18      jun     1029    1       -1      0       unknown
+3       38      technician      single  tertiary        no      -572    yes     yes     unknown 5       jun     26      24      -1      0       unknown
+4       34      technician      single  secondary       no      -476    yes     no      unknown 27      may     92      4       -1      0       unknown
+5       37      entrepreneur    married primary no      62      no      no      cellular        31      jul     404     2       -1      0       unknown
+6       43      services        married primary no      574     yes     no      cellular        8       may     140     1       -1      0       unknown
+7       54      technician      married secondary       no      324     yes     no      telephone       13      may     51      1       -1      0       unknown
+8       41      blue-collar     married secondary       no      121     yes     no      cellular        13      may     16      6       176     5       other
+9       52      housemaid       married primary no      1466    no      yes     cellular        20      nov     150     1       -1      0       unknown
+10      32      management      married secondary       no      6217    yes     yes     cellular        18      nov     486     2       181     2       failure
+```
+
+```sql
+select
+  id, 
+  array(age, job, marital, education, default, balance, housing, loan, contact, day, month, duration, campaign, pdays, previous, poutcome) as features
+from (
+  select 
+    quantify(
+      output_row, id, age, job, marital, education, default, balance, housing, loan, contact, day, month, duration, campaign, if(pdays==-1,0,pdays), previous, poutcome
+    ) as (id, age, job, marital, education, default, balance, housing, loan, contact, day, month, duration, campaign, pdays, previous, poutcome)
+  from (
+    select * from (
+      select
+        1 as train_first, false as output_row, id, age, job, marital, education, default, balance, housing, loan, contact, day, month, duration, campaign, pdays, previous, poutcome
+      from
+        train
+      union all
+      select
+        2 as train_first, true as output_row, id, age, job, marital, education, default, balance, housing, loan, contact, day, month, duration, campaign, pdays, previous, poutcome
+      from
+        test
+    ) t0
+    order by train_first, id asc
+  ) t1
+) t2
+limit 10;
+
+1       [30,2,1,2,0,1028,1,0,0,4,1,1294,2,0,0,0]
+2       [39,7,1,2,0,426,1,0,1,18,3,1029,1,0,0,0]
+3       [38,5,1,2,0,-572,0,1,1,5,3,26,24,0,0,0]
+4       [34,5,1,0,0,-476,0,0,1,27,5,92,4,0,0,0]
+5       [37,1,0,1,0,62,1,0,0,31,8,404,2,0,0,0]
+6       [43,4,0,1,0,574,0,0,0,8,5,140,1,0,0,0]
+7       [54,5,0,0,0,324,0,0,2,13,5,51,1,0,0,0]
+8       [41,0,0,0,0,121,0,0,0,13,5,16,6,176,5,3]
+9       [52,8,0,1,0,1466,1,1,0,20,9,150,1,0,0,0]
+10      [32,2,0,0,0,6217,0,1,0,18,9,486,2,181,2,1]
+```
\ No newline at end of file


[18/50] [abbrv] incubator-hivemall git commit: Updated license headers

Posted by my...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/model/SparseModel.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/model/SparseModel.java b/core/src/main/java/hivemall/model/SparseModel.java
index aaab869..96e1d5a 100644
--- a/core/src/main/java/hivemall/model/SparseModel.java
+++ b/core/src/main/java/hivemall/model/SparseModel.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.model;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/model/SynchronizedModelWrapper.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/model/SynchronizedModelWrapper.java b/core/src/main/java/hivemall/model/SynchronizedModelWrapper.java
index 99ee69c..67d05e5 100644
--- a/core/src/main/java/hivemall/model/SynchronizedModelWrapper.java
+++ b/core/src/main/java/hivemall/model/SynchronizedModelWrapper.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.model;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/model/WeightValue.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/model/WeightValue.java b/core/src/main/java/hivemall/model/WeightValue.java
index e6d98c6..4e19fef 100644
--- a/core/src/main/java/hivemall/model/WeightValue.java
+++ b/core/src/main/java/hivemall/model/WeightValue.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.model;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/model/WeightValueWithClock.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/model/WeightValueWithClock.java b/core/src/main/java/hivemall/model/WeightValueWithClock.java
index 249650a..e419c5d 100644
--- a/core/src/main/java/hivemall/model/WeightValueWithClock.java
+++ b/core/src/main/java/hivemall/model/WeightValueWithClock.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.model;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/regression/AROWRegressionUDTF.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/regression/AROWRegressionUDTF.java b/core/src/main/java/hivemall/regression/AROWRegressionUDTF.java
index b81a4bf..ac17e8b 100644
--- a/core/src/main/java/hivemall/regression/AROWRegressionUDTF.java
+++ b/core/src/main/java/hivemall/regression/AROWRegressionUDTF.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.regression;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/regression/AdaDeltaUDTF.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/regression/AdaDeltaUDTF.java b/core/src/main/java/hivemall/regression/AdaDeltaUDTF.java
index e807340..02473d9 100644
--- a/core/src/main/java/hivemall/regression/AdaDeltaUDTF.java
+++ b/core/src/main/java/hivemall/regression/AdaDeltaUDTF.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.regression;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/regression/AdaGradUDTF.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/regression/AdaGradUDTF.java b/core/src/main/java/hivemall/regression/AdaGradUDTF.java
index de48d97..01aec81 100644
--- a/core/src/main/java/hivemall/regression/AdaGradUDTF.java
+++ b/core/src/main/java/hivemall/regression/AdaGradUDTF.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.regression;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/regression/LogressUDTF.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/regression/LogressUDTF.java b/core/src/main/java/hivemall/regression/LogressUDTF.java
index ca3da71..956e8d9 100644
--- a/core/src/main/java/hivemall/regression/LogressUDTF.java
+++ b/core/src/main/java/hivemall/regression/LogressUDTF.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.regression;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/regression/PassiveAggressiveRegressionUDTF.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/regression/PassiveAggressiveRegressionUDTF.java b/core/src/main/java/hivemall/regression/PassiveAggressiveRegressionUDTF.java
index c089946..a293d51 100644
--- a/core/src/main/java/hivemall/regression/PassiveAggressiveRegressionUDTF.java
+++ b/core/src/main/java/hivemall/regression/PassiveAggressiveRegressionUDTF.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.regression;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/regression/RegressionBaseUDTF.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/regression/RegressionBaseUDTF.java b/core/src/main/java/hivemall/regression/RegressionBaseUDTF.java
index 561d4f7..a2ef4f7 100644
--- a/core/src/main/java/hivemall/regression/RegressionBaseUDTF.java
+++ b/core/src/main/java/hivemall/regression/RegressionBaseUDTF.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.regression;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/smile/ModelType.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/smile/ModelType.java b/core/src/main/java/hivemall/smile/ModelType.java
index 4405644..8925075 100644
--- a/core/src/main/java/hivemall/smile/ModelType.java
+++ b/core/src/main/java/hivemall/smile/ModelType.java
@@ -1,19 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.smile;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/smile/classification/DecisionTree.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/smile/classification/DecisionTree.java b/core/src/main/java/hivemall/smile/classification/DecisionTree.java
index 4f1c096..6b22473 100644
--- a/core/src/main/java/hivemall/smile/classification/DecisionTree.java
+++ b/core/src/main/java/hivemall/smile/classification/DecisionTree.java
@@ -1,19 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.
  */
 /*
  * Copyright (c) 2010 Haifeng Li

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/smile/classification/GradientTreeBoostingClassifierUDTF.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/smile/classification/GradientTreeBoostingClassifierUDTF.java b/core/src/main/java/hivemall/smile/classification/GradientTreeBoostingClassifierUDTF.java
index c9d229c..3a0924e 100644
--- a/core/src/main/java/hivemall/smile/classification/GradientTreeBoostingClassifierUDTF.java
+++ b/core/src/main/java/hivemall/smile/classification/GradientTreeBoostingClassifierUDTF.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.smile.classification;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/smile/classification/RandomForestClassifierUDTF.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/smile/classification/RandomForestClassifierUDTF.java b/core/src/main/java/hivemall/smile/classification/RandomForestClassifierUDTF.java
index 165e513..03db65c 100644
--- a/core/src/main/java/hivemall/smile/classification/RandomForestClassifierUDTF.java
+++ b/core/src/main/java/hivemall/smile/classification/RandomForestClassifierUDTF.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.smile.classification;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/smile/data/Attribute.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/smile/data/Attribute.java b/core/src/main/java/hivemall/smile/data/Attribute.java
index 9d06662..be6651a 100644
--- a/core/src/main/java/hivemall/smile/data/Attribute.java
+++ b/core/src/main/java/hivemall/smile/data/Attribute.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.smile.data;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/smile/regression/RandomForestRegressionUDTF.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/smile/regression/RandomForestRegressionUDTF.java b/core/src/main/java/hivemall/smile/regression/RandomForestRegressionUDTF.java
index d61b8df..ebb58c6 100644
--- a/core/src/main/java/hivemall/smile/regression/RandomForestRegressionUDTF.java
+++ b/core/src/main/java/hivemall/smile/regression/RandomForestRegressionUDTF.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.smile.regression;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/smile/regression/RegressionTree.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/smile/regression/RegressionTree.java b/core/src/main/java/hivemall/smile/regression/RegressionTree.java
index f91ab4a..07887c1 100755
--- a/core/src/main/java/hivemall/smile/regression/RegressionTree.java
+++ b/core/src/main/java/hivemall/smile/regression/RegressionTree.java
@@ -1,19 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.
  */
 /*
  * Copyright (c) 2010 Haifeng Li

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/smile/tools/GuessAttributesUDF.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/smile/tools/GuessAttributesUDF.java b/core/src/main/java/hivemall/smile/tools/GuessAttributesUDF.java
index 0928061..bc2794e 100644
--- a/core/src/main/java/hivemall/smile/tools/GuessAttributesUDF.java
+++ b/core/src/main/java/hivemall/smile/tools/GuessAttributesUDF.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.smile.tools;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/smile/tools/RandomForestEnsembleUDAF.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/smile/tools/RandomForestEnsembleUDAF.java b/core/src/main/java/hivemall/smile/tools/RandomForestEnsembleUDAF.java
index 9fce421..7fd841a 100644
--- a/core/src/main/java/hivemall/smile/tools/RandomForestEnsembleUDAF.java
+++ b/core/src/main/java/hivemall/smile/tools/RandomForestEnsembleUDAF.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.smile.tools;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/smile/tools/TreePredictUDF.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/smile/tools/TreePredictUDF.java b/core/src/main/java/hivemall/smile/tools/TreePredictUDF.java
index 3228517..b5a81d4 100644
--- a/core/src/main/java/hivemall/smile/tools/TreePredictUDF.java
+++ b/core/src/main/java/hivemall/smile/tools/TreePredictUDF.java
@@ -1,19 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.smile.tools;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/smile/utils/SmileExtUtils.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/smile/utils/SmileExtUtils.java b/core/src/main/java/hivemall/smile/utils/SmileExtUtils.java
index cfd20d4..c0dfc1c 100644
--- a/core/src/main/java/hivemall/smile/utils/SmileExtUtils.java
+++ b/core/src/main/java/hivemall/smile/utils/SmileExtUtils.java
@@ -1,19 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.smile.utils;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/smile/utils/SmileTaskExecutor.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/smile/utils/SmileTaskExecutor.java b/core/src/main/java/hivemall/smile/utils/SmileTaskExecutor.java
index 6e9f39b..cbb0f04 100644
--- a/core/src/main/java/hivemall/smile/utils/SmileTaskExecutor.java
+++ b/core/src/main/java/hivemall/smile/utils/SmileTaskExecutor.java
@@ -1,19 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.smile.utils;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/smile/vm/Operation.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/smile/vm/Operation.java b/core/src/main/java/hivemall/smile/vm/Operation.java
index 1c08e72..fff617f 100644
--- a/core/src/main/java/hivemall/smile/vm/Operation.java
+++ b/core/src/main/java/hivemall/smile/vm/Operation.java
@@ -1,19 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.smile.vm;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/smile/vm/StackMachine.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/smile/vm/StackMachine.java b/core/src/main/java/hivemall/smile/vm/StackMachine.java
index 0becc98..3bf8b46 100644
--- a/core/src/main/java/hivemall/smile/vm/StackMachine.java
+++ b/core/src/main/java/hivemall/smile/vm/StackMachine.java
@@ -1,19 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.smile.vm;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/smile/vm/VMRuntimeException.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/smile/vm/VMRuntimeException.java b/core/src/main/java/hivemall/smile/vm/VMRuntimeException.java
index 4139707..7fc89c8 100644
--- a/core/src/main/java/hivemall/smile/vm/VMRuntimeException.java
+++ b/core/src/main/java/hivemall/smile/vm/VMRuntimeException.java
@@ -1,19 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.smile.vm;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/tools/ConvertLabelUDF.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/tools/ConvertLabelUDF.java b/core/src/main/java/hivemall/tools/ConvertLabelUDF.java
index 1f1b255..4c8a739 100644
--- a/core/src/main/java/hivemall/tools/ConvertLabelUDF.java
+++ b/core/src/main/java/hivemall/tools/ConvertLabelUDF.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.tools;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/tools/EachTopKUDTF.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/tools/EachTopKUDTF.java b/core/src/main/java/hivemall/tools/EachTopKUDTF.java
index 9451575..da33eb6 100644
--- a/core/src/main/java/hivemall/tools/EachTopKUDTF.java
+++ b/core/src/main/java/hivemall/tools/EachTopKUDTF.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.tools;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/tools/GenerateSeriesUDTF.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/tools/GenerateSeriesUDTF.java b/core/src/main/java/hivemall/tools/GenerateSeriesUDTF.java
index 50905bd..3792605 100644
--- a/core/src/main/java/hivemall/tools/GenerateSeriesUDTF.java
+++ b/core/src/main/java/hivemall/tools/GenerateSeriesUDTF.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.tools;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/tools/RankSequenceUDF.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/tools/RankSequenceUDF.java b/core/src/main/java/hivemall/tools/RankSequenceUDF.java
index bc51018..993aa87 100644
--- a/core/src/main/java/hivemall/tools/RankSequenceUDF.java
+++ b/core/src/main/java/hivemall/tools/RankSequenceUDF.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.tools;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/tools/array/AllocFloatArrayUDF.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/tools/array/AllocFloatArrayUDF.java b/core/src/main/java/hivemall/tools/array/AllocFloatArrayUDF.java
index c36fa68..45cb2cd 100644
--- a/core/src/main/java/hivemall/tools/array/AllocFloatArrayUDF.java
+++ b/core/src/main/java/hivemall/tools/array/AllocFloatArrayUDF.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.tools.array;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/tools/array/ArrayAvgGenericUDAF.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/tools/array/ArrayAvgGenericUDAF.java b/core/src/main/java/hivemall/tools/array/ArrayAvgGenericUDAF.java
index c21b0ee..a2e3e55 100644
--- a/core/src/main/java/hivemall/tools/array/ArrayAvgGenericUDAF.java
+++ b/core/src/main/java/hivemall/tools/array/ArrayAvgGenericUDAF.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.tools.array;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/tools/array/ArrayConcatUDF.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/tools/array/ArrayConcatUDF.java b/core/src/main/java/hivemall/tools/array/ArrayConcatUDF.java
index 197369a..ad0dcc3 100644
--- a/core/src/main/java/hivemall/tools/array/ArrayConcatUDF.java
+++ b/core/src/main/java/hivemall/tools/array/ArrayConcatUDF.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.tools.array;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/tools/array/ArrayIntersectUDF.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/tools/array/ArrayIntersectUDF.java b/core/src/main/java/hivemall/tools/array/ArrayIntersectUDF.java
index 7e6f588..de89f35 100644
--- a/core/src/main/java/hivemall/tools/array/ArrayIntersectUDF.java
+++ b/core/src/main/java/hivemall/tools/array/ArrayIntersectUDF.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.tools.array;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/tools/array/ArrayRemoveUDF.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/tools/array/ArrayRemoveUDF.java b/core/src/main/java/hivemall/tools/array/ArrayRemoveUDF.java
index 9eb356c..15a2c3d 100644
--- a/core/src/main/java/hivemall/tools/array/ArrayRemoveUDF.java
+++ b/core/src/main/java/hivemall/tools/array/ArrayRemoveUDF.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.tools.array;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/tools/array/ArraySumUDAF.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/tools/array/ArraySumUDAF.java b/core/src/main/java/hivemall/tools/array/ArraySumUDAF.java
index f123c2e..9d4fc93 100644
--- a/core/src/main/java/hivemall/tools/array/ArraySumUDAF.java
+++ b/core/src/main/java/hivemall/tools/array/ArraySumUDAF.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.tools.array;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/tools/array/CollectAllUDAF.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/tools/array/CollectAllUDAF.java b/core/src/main/java/hivemall/tools/array/CollectAllUDAF.java
index 57aff1b..6ac48b1 100644
--- a/core/src/main/java/hivemall/tools/array/CollectAllUDAF.java
+++ b/core/src/main/java/hivemall/tools/array/CollectAllUDAF.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.tools.array;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/tools/array/SortAndUniqArrayUDF.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/tools/array/SortAndUniqArrayUDF.java b/core/src/main/java/hivemall/tools/array/SortAndUniqArrayUDF.java
index 0ff1d94..844bb9f 100644
--- a/core/src/main/java/hivemall/tools/array/SortAndUniqArrayUDF.java
+++ b/core/src/main/java/hivemall/tools/array/SortAndUniqArrayUDF.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.tools.array;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/tools/array/SubarrayEndWithUDF.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/tools/array/SubarrayEndWithUDF.java b/core/src/main/java/hivemall/tools/array/SubarrayEndWithUDF.java
index 428d4be..ecc8a05 100644
--- a/core/src/main/java/hivemall/tools/array/SubarrayEndWithUDF.java
+++ b/core/src/main/java/hivemall/tools/array/SubarrayEndWithUDF.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.tools.array;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/tools/array/SubarrayStartWithUDF.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/tools/array/SubarrayStartWithUDF.java b/core/src/main/java/hivemall/tools/array/SubarrayStartWithUDF.java
index a0b77ee..3f7266d 100644
--- a/core/src/main/java/hivemall/tools/array/SubarrayStartWithUDF.java
+++ b/core/src/main/java/hivemall/tools/array/SubarrayStartWithUDF.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.tools.array;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/tools/array/SubarrayUDF.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/tools/array/SubarrayUDF.java b/core/src/main/java/hivemall/tools/array/SubarrayUDF.java
index ba6cb85..0b63a93 100644
--- a/core/src/main/java/hivemall/tools/array/SubarrayUDF.java
+++ b/core/src/main/java/hivemall/tools/array/SubarrayUDF.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.tools.array;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/tools/array/ToStringArrayUDF.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/tools/array/ToStringArrayUDF.java b/core/src/main/java/hivemall/tools/array/ToStringArrayUDF.java
index d57fd33..306673c 100644
--- a/core/src/main/java/hivemall/tools/array/ToStringArrayUDF.java
+++ b/core/src/main/java/hivemall/tools/array/ToStringArrayUDF.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.tools.array;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/tools/bits/BitsCollectUDAF.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/tools/bits/BitsCollectUDAF.java b/core/src/main/java/hivemall/tools/bits/BitsCollectUDAF.java
index 93cbfa2..cc865a0 100644
--- a/core/src/main/java/hivemall/tools/bits/BitsCollectUDAF.java
+++ b/core/src/main/java/hivemall/tools/bits/BitsCollectUDAF.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.tools.bits;
 



[19/50] [abbrv] incubator-hivemall git commit: Updated license headers

Posted by my...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/knn/distance/JaccardDistanceUDF.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/knn/distance/JaccardDistanceUDF.java b/core/src/main/java/hivemall/knn/distance/JaccardDistanceUDF.java
index 5eaa393..70c281e 100644
--- a/core/src/main/java/hivemall/knn/distance/JaccardDistanceUDF.java
+++ b/core/src/main/java/hivemall/knn/distance/JaccardDistanceUDF.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.knn.distance;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/knn/distance/KLDivergenceUDF.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/knn/distance/KLDivergenceUDF.java b/core/src/main/java/hivemall/knn/distance/KLDivergenceUDF.java
index efa2ccf..9ba682a 100644
--- a/core/src/main/java/hivemall/knn/distance/KLDivergenceUDF.java
+++ b/core/src/main/java/hivemall/knn/distance/KLDivergenceUDF.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.knn.distance;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/knn/distance/ManhattanDistanceUDF.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/knn/distance/ManhattanDistanceUDF.java b/core/src/main/java/hivemall/knn/distance/ManhattanDistanceUDF.java
index acb9296..4ab258d 100644
--- a/core/src/main/java/hivemall/knn/distance/ManhattanDistanceUDF.java
+++ b/core/src/main/java/hivemall/knn/distance/ManhattanDistanceUDF.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.knn.distance;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/knn/distance/MinkowskiDistanceUDF.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/knn/distance/MinkowskiDistanceUDF.java b/core/src/main/java/hivemall/knn/distance/MinkowskiDistanceUDF.java
index 098540c..b0282b8 100644
--- a/core/src/main/java/hivemall/knn/distance/MinkowskiDistanceUDF.java
+++ b/core/src/main/java/hivemall/knn/distance/MinkowskiDistanceUDF.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.knn.distance;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/knn/distance/PopcountUDF.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/knn/distance/PopcountUDF.java b/core/src/main/java/hivemall/knn/distance/PopcountUDF.java
index fc006c5..0d99d4d 100644
--- a/core/src/main/java/hivemall/knn/distance/PopcountUDF.java
+++ b/core/src/main/java/hivemall/knn/distance/PopcountUDF.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.knn.distance;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/knn/lsh/MinHashUDTF.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/knn/lsh/MinHashUDTF.java b/core/src/main/java/hivemall/knn/lsh/MinHashUDTF.java
index ee7d099..bf4a8fb 100644
--- a/core/src/main/java/hivemall/knn/lsh/MinHashUDTF.java
+++ b/core/src/main/java/hivemall/knn/lsh/MinHashUDTF.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.knn.lsh;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/knn/lsh/MinHashesUDF.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/knn/lsh/MinHashesUDF.java b/core/src/main/java/hivemall/knn/lsh/MinHashesUDF.java
index cc2140e..6fc12c9 100644
--- a/core/src/main/java/hivemall/knn/lsh/MinHashesUDF.java
+++ b/core/src/main/java/hivemall/knn/lsh/MinHashesUDF.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.knn.lsh;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/knn/lsh/bBitMinHashUDF.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/knn/lsh/bBitMinHashUDF.java b/core/src/main/java/hivemall/knn/lsh/bBitMinHashUDF.java
index 00603d6..71f4fbe 100644
--- a/core/src/main/java/hivemall/knn/lsh/bBitMinHashUDF.java
+++ b/core/src/main/java/hivemall/knn/lsh/bBitMinHashUDF.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.knn.lsh;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/knn/similarity/AngularSimilarityUDF.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/knn/similarity/AngularSimilarityUDF.java b/core/src/main/java/hivemall/knn/similarity/AngularSimilarityUDF.java
index 2763a48..1e881fd 100644
--- a/core/src/main/java/hivemall/knn/similarity/AngularSimilarityUDF.java
+++ b/core/src/main/java/hivemall/knn/similarity/AngularSimilarityUDF.java
@@ -1,3 +1,21 @@
+/*
+ * 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.knn.similarity;
 
 import hivemall.utils.hadoop.HiveUtils;

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/knn/similarity/CosineSimilarityUDF.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/knn/similarity/CosineSimilarityUDF.java b/core/src/main/java/hivemall/knn/similarity/CosineSimilarityUDF.java
index 9fb73d3..f907abe 100644
--- a/core/src/main/java/hivemall/knn/similarity/CosineSimilarityUDF.java
+++ b/core/src/main/java/hivemall/knn/similarity/CosineSimilarityUDF.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.knn.similarity;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/knn/similarity/Distance2SimilarityUDF.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/knn/similarity/Distance2SimilarityUDF.java b/core/src/main/java/hivemall/knn/similarity/Distance2SimilarityUDF.java
index 15f0a0f..43c42a7 100644
--- a/core/src/main/java/hivemall/knn/similarity/Distance2SimilarityUDF.java
+++ b/core/src/main/java/hivemall/knn/similarity/Distance2SimilarityUDF.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.knn.similarity;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/knn/similarity/EuclidSimilarity.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/knn/similarity/EuclidSimilarity.java b/core/src/main/java/hivemall/knn/similarity/EuclidSimilarity.java
index ab6ff67..fad06a5 100644
--- a/core/src/main/java/hivemall/knn/similarity/EuclidSimilarity.java
+++ b/core/src/main/java/hivemall/knn/similarity/EuclidSimilarity.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.knn.similarity;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/knn/similarity/JaccardIndexUDF.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/knn/similarity/JaccardIndexUDF.java b/core/src/main/java/hivemall/knn/similarity/JaccardIndexUDF.java
index 00e9499..f453f82 100644
--- a/core/src/main/java/hivemall/knn/similarity/JaccardIndexUDF.java
+++ b/core/src/main/java/hivemall/knn/similarity/JaccardIndexUDF.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.knn.similarity;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/mf/BPRMFPredictionUDF.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/mf/BPRMFPredictionUDF.java b/core/src/main/java/hivemall/mf/BPRMFPredictionUDF.java
index 1a2b2a2..e555d7b 100644
--- a/core/src/main/java/hivemall/mf/BPRMFPredictionUDF.java
+++ b/core/src/main/java/hivemall/mf/BPRMFPredictionUDF.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.mf;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/mf/BPRMatrixFactorizationUDTF.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/mf/BPRMatrixFactorizationUDTF.java b/core/src/main/java/hivemall/mf/BPRMatrixFactorizationUDTF.java
index d859f29..714e0b4 100644
--- a/core/src/main/java/hivemall/mf/BPRMatrixFactorizationUDTF.java
+++ b/core/src/main/java/hivemall/mf/BPRMatrixFactorizationUDTF.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.mf;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/mf/FactorizedModel.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/mf/FactorizedModel.java b/core/src/main/java/hivemall/mf/FactorizedModel.java
index 06b40ee..b92a5d8 100644
--- a/core/src/main/java/hivemall/mf/FactorizedModel.java
+++ b/core/src/main/java/hivemall/mf/FactorizedModel.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.mf;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/mf/MFPredictionUDF.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/mf/MFPredictionUDF.java b/core/src/main/java/hivemall/mf/MFPredictionUDF.java
index ee6627c..f2fd0e2 100644
--- a/core/src/main/java/hivemall/mf/MFPredictionUDF.java
+++ b/core/src/main/java/hivemall/mf/MFPredictionUDF.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.mf;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/mf/MatrixFactorizationAdaGradUDTF.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/mf/MatrixFactorizationAdaGradUDTF.java b/core/src/main/java/hivemall/mf/MatrixFactorizationAdaGradUDTF.java
index bb066ba..82919c1 100644
--- a/core/src/main/java/hivemall/mf/MatrixFactorizationAdaGradUDTF.java
+++ b/core/src/main/java/hivemall/mf/MatrixFactorizationAdaGradUDTF.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.mf;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/mf/MatrixFactorizationSGDUDTF.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/mf/MatrixFactorizationSGDUDTF.java b/core/src/main/java/hivemall/mf/MatrixFactorizationSGDUDTF.java
index 317da85..a95d01d 100644
--- a/core/src/main/java/hivemall/mf/MatrixFactorizationSGDUDTF.java
+++ b/core/src/main/java/hivemall/mf/MatrixFactorizationSGDUDTF.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.mf;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/mf/OnlineMatrixFactorizationUDTF.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/mf/OnlineMatrixFactorizationUDTF.java b/core/src/main/java/hivemall/mf/OnlineMatrixFactorizationUDTF.java
index 742e578..bfc1f19 100644
--- a/core/src/main/java/hivemall/mf/OnlineMatrixFactorizationUDTF.java
+++ b/core/src/main/java/hivemall/mf/OnlineMatrixFactorizationUDTF.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.mf;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/mf/Rating.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/mf/Rating.java b/core/src/main/java/hivemall/mf/Rating.java
index f5c7d4d..67ca302 100644
--- a/core/src/main/java/hivemall/mf/Rating.java
+++ b/core/src/main/java/hivemall/mf/Rating.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.mf;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/mf/RatingInitilizer.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/mf/RatingInitilizer.java b/core/src/main/java/hivemall/mf/RatingInitilizer.java
index cd73209..1938b00 100644
--- a/core/src/main/java/hivemall/mf/RatingInitilizer.java
+++ b/core/src/main/java/hivemall/mf/RatingInitilizer.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.mf;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/mix/MixEnv.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/mix/MixEnv.java b/core/src/main/java/hivemall/mix/MixEnv.java
index d6dc63b..04fc140 100644
--- a/core/src/main/java/hivemall/mix/MixEnv.java
+++ b/core/src/main/java/hivemall/mix/MixEnv.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.mix;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/mix/MixMessage.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/mix/MixMessage.java b/core/src/main/java/hivemall/mix/MixMessage.java
index d827bf5..9fd3676 100644
--- a/core/src/main/java/hivemall/mix/MixMessage.java
+++ b/core/src/main/java/hivemall/mix/MixMessage.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.mix;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/mix/MixMessageDecoder.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/mix/MixMessageDecoder.java b/core/src/main/java/hivemall/mix/MixMessageDecoder.java
index 7b59774..0841fca 100644
--- a/core/src/main/java/hivemall/mix/MixMessageDecoder.java
+++ b/core/src/main/java/hivemall/mix/MixMessageDecoder.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.mix;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/mix/MixMessageEncoder.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/mix/MixMessageEncoder.java b/core/src/main/java/hivemall/mix/MixMessageEncoder.java
index 1efb1d3..c295e1c 100644
--- a/core/src/main/java/hivemall/mix/MixMessageEncoder.java
+++ b/core/src/main/java/hivemall/mix/MixMessageEncoder.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.mix;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/mix/MixedModel.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/mix/MixedModel.java b/core/src/main/java/hivemall/mix/MixedModel.java
index d645bbe..0bb1fe2 100644
--- a/core/src/main/java/hivemall/mix/MixedModel.java
+++ b/core/src/main/java/hivemall/mix/MixedModel.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.mix;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/mix/MixedWeight.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/mix/MixedWeight.java b/core/src/main/java/hivemall/mix/MixedWeight.java
index b059e2b..34ef2da 100644
--- a/core/src/main/java/hivemall/mix/MixedWeight.java
+++ b/core/src/main/java/hivemall/mix/MixedWeight.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.mix;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/mix/NodeInfo.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/mix/NodeInfo.java b/core/src/main/java/hivemall/mix/NodeInfo.java
index fe7bcfc..20e8fbd 100644
--- a/core/src/main/java/hivemall/mix/NodeInfo.java
+++ b/core/src/main/java/hivemall/mix/NodeInfo.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.mix;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/mix/client/MixClient.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/mix/client/MixClient.java b/core/src/main/java/hivemall/mix/client/MixClient.java
index 97570c6..05b6d29 100644
--- a/core/src/main/java/hivemall/mix/client/MixClient.java
+++ b/core/src/main/java/hivemall/mix/client/MixClient.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.mix.client;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/mix/client/MixClientHandler.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/mix/client/MixClientHandler.java b/core/src/main/java/hivemall/mix/client/MixClientHandler.java
index a37e6c4..8608c43 100644
--- a/core/src/main/java/hivemall/mix/client/MixClientHandler.java
+++ b/core/src/main/java/hivemall/mix/client/MixClientHandler.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.mix.client;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/mix/client/MixClientInitializer.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/mix/client/MixClientInitializer.java b/core/src/main/java/hivemall/mix/client/MixClientInitializer.java
index 6226e03..fb9a24c 100644
--- a/core/src/main/java/hivemall/mix/client/MixClientInitializer.java
+++ b/core/src/main/java/hivemall/mix/client/MixClientInitializer.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.mix.client;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/mix/client/MixRequestRouter.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/mix/client/MixRequestRouter.java b/core/src/main/java/hivemall/mix/client/MixRequestRouter.java
index 7fe0361..24cfb0c 100644
--- a/core/src/main/java/hivemall/mix/client/MixRequestRouter.java
+++ b/core/src/main/java/hivemall/mix/client/MixRequestRouter.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.mix.client;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/model/AbstractPredictionModel.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/model/AbstractPredictionModel.java b/core/src/main/java/hivemall/model/AbstractPredictionModel.java
index 112f64f..37b69da 100644
--- a/core/src/main/java/hivemall/model/AbstractPredictionModel.java
+++ b/core/src/main/java/hivemall/model/AbstractPredictionModel.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.model;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/model/DenseModel.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/model/DenseModel.java b/core/src/main/java/hivemall/model/DenseModel.java
index ee57574..628b43e 100644
--- a/core/src/main/java/hivemall/model/DenseModel.java
+++ b/core/src/main/java/hivemall/model/DenseModel.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.model;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/model/FeatureValue.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/model/FeatureValue.java b/core/src/main/java/hivemall/model/FeatureValue.java
index 6e0f051..8f2b728 100644
--- a/core/src/main/java/hivemall/model/FeatureValue.java
+++ b/core/src/main/java/hivemall/model/FeatureValue.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.model;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/model/IWeightValue.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/model/IWeightValue.java b/core/src/main/java/hivemall/model/IWeightValue.java
index 988e4a1..cd25564 100644
--- a/core/src/main/java/hivemall/model/IWeightValue.java
+++ b/core/src/main/java/hivemall/model/IWeightValue.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.model;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/model/Margin.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/model/Margin.java b/core/src/main/java/hivemall/model/Margin.java
index 98c91d7..8db854f 100644
--- a/core/src/main/java/hivemall/model/Margin.java
+++ b/core/src/main/java/hivemall/model/Margin.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.model;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/model/ModelUpdateHandler.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/model/ModelUpdateHandler.java b/core/src/main/java/hivemall/model/ModelUpdateHandler.java
index 4a5e44a..6882947 100644
--- a/core/src/main/java/hivemall/model/ModelUpdateHandler.java
+++ b/core/src/main/java/hivemall/model/ModelUpdateHandler.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.model;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/model/PredictionModel.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/model/PredictionModel.java b/core/src/main/java/hivemall/model/PredictionModel.java
index a8efee0..71f67d5 100644
--- a/core/src/main/java/hivemall/model/PredictionModel.java
+++ b/core/src/main/java/hivemall/model/PredictionModel.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.model;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/model/PredictionResult.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/model/PredictionResult.java b/core/src/main/java/hivemall/model/PredictionResult.java
index 09d042b..5d78f85 100644
--- a/core/src/main/java/hivemall/model/PredictionResult.java
+++ b/core/src/main/java/hivemall/model/PredictionResult.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.model;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/model/SpaceEfficientDenseModel.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/model/SpaceEfficientDenseModel.java b/core/src/main/java/hivemall/model/SpaceEfficientDenseModel.java
index b3cd3ff..46f5d6e 100644
--- a/core/src/main/java/hivemall/model/SpaceEfficientDenseModel.java
+++ b/core/src/main/java/hivemall/model/SpaceEfficientDenseModel.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.model;
 


[41/50] [abbrv] incubator-hivemall git commit: Fixed layouts

Posted by my...@apache.org.
Fixed layouts


Project: http://git-wip-us.apache.org/repos/asf/incubator-hivemall/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-hivemall/commit/caeafaee
Tree: http://git-wip-us.apache.org/repos/asf/incubator-hivemall/tree/caeafaee
Diff: http://git-wip-us.apache.org/repos/asf/incubator-hivemall/diff/caeafaee

Branch: refs/heads/master
Commit: caeafaee9b462e4de4de4b42a87fd0da6fb381c9
Parents: 52bc44f
Author: myui <yu...@gmail.com>
Authored: Mon Nov 14 20:13:53 2016 +0900
Committer: myui <yu...@gmail.com>
Committed: Mon Nov 14 20:13:53 2016 +0900

----------------------------------------------------------------------
 docs/gitbook/getting_started/permanent-functions.md | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/caeafaee/docs/gitbook/getting_started/permanent-functions.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/getting_started/permanent-functions.md b/docs/gitbook/getting_started/permanent-functions.md
index a4879c3..75156fe 100644
--- a/docs/gitbook/getting_started/permanent-functions.md
+++ b/docs/gitbook/getting_started/permanent-functions.md
@@ -58,4 +58,4 @@ show functions "hivemall.*";
 ```
 
 > #### Caution
-You need to specify "hivemall." prefix to call hivemall UDFs in your queries if UDFs are loaded into non-default scheme, in this case _hivemall_.
\ No newline at end of file
+You need to specify "hivemall." prefix to call hivemall UDFs in your queries if UDFs are loaded into non-default scheme, in this case _hivemall_.


[48/50] [abbrv] incubator-hivemall git commit: Update README.md

Posted by my...@apache.org.
Update README.md

Added twitter button

Project: http://git-wip-us.apache.org/repos/asf/incubator-hivemall/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-hivemall/commit/3e964700
Tree: http://git-wip-us.apache.org/repos/asf/incubator-hivemall/tree/3e964700
Diff: http://git-wip-us.apache.org/repos/asf/incubator-hivemall/diff/3e964700

Branch: refs/heads/master
Commit: 3e964700a9e9032102a82f2afa0dbd978ed43019
Parents: effe461
Author: Makoto YUI <yu...@gmail.com>
Authored: Mon Nov 21 17:06:16 2016 +0900
Committer: GitHub <no...@github.com>
Committed: Mon Nov 21 17:06:16 2016 +0900

----------------------------------------------------------------------
 README.md | 1 +
 1 file changed, 1 insertion(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/3e964700/README.md
----------------------------------------------------------------------
diff --git a/README.md b/README.md
index 2841ef7..7a0978b 100644
--- a/README.md
+++ b/README.md
@@ -24,6 +24,7 @@ Apache Hivemall: Hive scalable machine learning library
 [![Documentation Status](https://readthedocs.org/projects/hivemall-docs/badge/?version=latest)](https://readthedocs.org/projects/hivemall-docs/?badge=latest)
 [![License](http://img.shields.io/:license-Apache_v2-blue.svg)](https://github.com/myui/hivemall/blob/master/LICENSE)
 [![Coverage Status](https://coveralls.io/repos/github/myui/hivemall/badge.svg)](https://coveralls.io/github/myui/hivemall)
+[![Twitter Follow](https://img.shields.io/twitter/follow/ApacheHivemall.svg?style=social&label=Follow)](https://twitter.com/ApacheHivemall)
 
 **News:** Hivemall joins [Apache Incubator](http://hivemall.incubator.apache.org/)! :tada: Currently in the process of moving the project repository to ASF. This repository will be deprecated soon. Please move your stars and forks to [the new repository](https://github.com/apache/incubator-hivemall).
 


[38/50] [abbrv] incubator-hivemall git commit: Updated copyrights

Posted by my...@apache.org.
Updated copyrights

Project: http://git-wip-us.apache.org/repos/asf/incubator-hivemall/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-hivemall/commit/fd872c24
Tree: http://git-wip-us.apache.org/repos/asf/incubator-hivemall/tree/fd872c24
Diff: http://git-wip-us.apache.org/repos/asf/incubator-hivemall/diff/fd872c24

Branch: refs/heads/master
Commit: fd872c245ecefd14f117c927acf5c3eb4ee000d2
Parents: 190cb6e
Author: myui <yu...@gmail.com>
Authored: Sat Nov 12 16:20:37 2016 +0900
Committer: myui <yu...@gmail.com>
Committed: Sat Nov 12 16:20:37 2016 +0900

----------------------------------------------------------------------
 NOTICE | 1 +
 1 file changed, 1 insertion(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/fd872c24/NOTICE
----------------------------------------------------------------------
diff --git a/NOTICE b/NOTICE
index c65fa03..2f22827 100644
--- a/NOTICE
+++ b/NOTICE
@@ -6,6 +6,7 @@ The Apache Software Foundation (http://www.apache.org/).
 
 Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
 Copyright (C) 2015-2016 Makoto Yui
+Copyright (C) 2015-2016 Treasure Data, Inc.
 
 ------------------------------------------------------------------------------------------------------
 Copyright notifications which have been relocated from source files


[44/50] [abbrv] incubator-hivemall git commit: Updated the userguide

Posted by my...@apache.org.
Updated the userguide

Project: http://git-wip-us.apache.org/repos/asf/incubator-hivemall/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-hivemall/commit/a71bbb75
Tree: http://git-wip-us.apache.org/repos/asf/incubator-hivemall/tree/a71bbb75
Diff: http://git-wip-us.apache.org/repos/asf/incubator-hivemall/diff/a71bbb75

Branch: refs/heads/master
Commit: a71bbb75b3d6dc9d820ccf33939eb17d1de51d43
Parents: ae2307f
Author: myui <yu...@gmail.com>
Authored: Thu Nov 17 21:16:14 2016 +0900
Committer: myui <yu...@gmail.com>
Committed: Thu Nov 17 23:40:56 2016 +0900

----------------------------------------------------------------------
 docs/gitbook/SUMMARY.md                         |   2 +
 docs/gitbook/anomaly/lof.md                     |  16 +-
 docs/gitbook/binaryclass/a9a_lr.md              | 187 ++++++-----
 docs/gitbook/binaryclass/a9a_minibatch.md       |   7 +-
 docs/gitbook/binaryclass/kdd2010a_dataset.md    |   6 +-
 docs/gitbook/binaryclass/kdd2010b_dataset.md    |   6 +-
 docs/gitbook/binaryclass/news20_scw.md          |   2 +-
 docs/gitbook/binaryclass/titanic_rf.md          | 318 +++++++++++++++++++
 docs/gitbook/binaryclass/webspam_scw.md         |   2 +-
 docs/gitbook/eval/lr_datagen.md                 |   6 +-
 docs/gitbook/eval/stat_eval.md                  |  10 +-
 docs/gitbook/ft_engineering/hashing.md          |   4 +-
 docs/gitbook/getting_started/input-format.md    |  14 +-
 .../getting_started/permanent-functions.md      |   5 +-
 docs/gitbook/misc/generic_funcs.md              | 203 ++++++------
 docs/gitbook/misc/topk.md                       |  17 +-
 docs/gitbook/multiclass/iris_dataset.md         |   2 +-
 docs/gitbook/multiclass/iris_randomforest.md    |   4 +-
 docs/gitbook/multiclass/iris_scw.md             |   2 +-
 docs/gitbook/multiclass/news20_scw.md           |   2 +-
 docs/gitbook/recommend/item_based_cf.md         |   4 +-
 docs/gitbook/recommend/movielens_fm.md          |   7 +-
 docs/gitbook/recommend/movielens_mf.md          |  20 +-
 docs/gitbook/recommend/news20_knn.md            |   2 +-
 docs/gitbook/regression/e2006_arow.md           |   2 +-
 docs/gitbook/regression/kddcup12tr2_adagrad.md  | 254 +++++++--------
 docs/gitbook/regression/kddcup12tr2_dataset.md  |   2 +-
 .../regression/kddcup12tr2_lr_amplify.md        |   6 +-
 .../resources/images/kddtrack2tables.png        | Bin 0 -> 30323 bytes
 docs/gitbook/tips/addbias.md                    |   2 +-
 docs/gitbook/tips/emr.md                        |   2 +
 docs/gitbook/tips/hadoop_tuning.md              |   2 +
 docs/gitbook/tips/mixserver.md                  | 169 +++++-----
 docs/gitbook/tips/rand_amplify.md               |  12 +-
 docs/gitbook/tips/rowid.md                      |  27 +-
 docs/gitbook/tips/rt_prediction.md              |  16 +-
 36 files changed, 834 insertions(+), 508 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/a71bbb75/docs/gitbook/SUMMARY.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/SUMMARY.md b/docs/gitbook/SUMMARY.md
index 7ef1b9b..c333c98 100644
--- a/docs/gitbook/SUMMARY.md
+++ b/docs/gitbook/SUMMARY.md
@@ -92,6 +92,8 @@
 * [Webspam Tutorial](binaryclass/webspam.md)
     * [Data pareparation](binaryclass/webspam_dataset.md)
     * [PA1, AROW, SCW](binaryclass/webspam_scw.md)
+
+* [Kaggle Titanic Tutorial](binaryclass/titanic_rf.md)
     
 ## Part VI - Multiclass classification
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/a71bbb75/docs/gitbook/anomaly/lof.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/anomaly/lof.md b/docs/gitbook/anomaly/lof.md
index 48990f8..39a6e9f 100644
--- a/docs/gitbook/anomaly/lof.md
+++ b/docs/gitbook/anomaly/lof.md
@@ -19,6 +19,8 @@
         
 This article introduce how to find outliers using [Local Outlier Detection (LOF)](http://en.wikipedia.org/wiki/Local_outlier_factor) on Hivemall.
 
+<!-- toc -->
+
 # Data Preparation
 
 ```sql
@@ -36,9 +38,9 @@ ROW FORMAT DELIMITED
 STORED AS TEXTFILE LOCATION '/dataset/lof/hundred_balls';
 ```
 
-Download [hundred_balls.txt](https://github.com/myui/hivemall/blob/master/resources/examples/lof/hundred_balls.txt) that is originally provides in [this article](http://next.rikunabi.com/tech/docs/ct_s03600.jsp?p=002259).
+Download [hundred_balls.txt](https://gist.githubusercontent.com/myui/f8b44ab925bc198e6d11b18fdd21269d/raw/bed05f811e4c351ed959e0159405690f2f11e577/hundred_balls.txt) that is originally provides in [this article](http://next.rikunabi.com/tech/docs/ct_s03600.jsp?p=002259).
 
-You can find outliers in [this picture](http://next.rikunabi.com/tech/contents/ts_report/img/201303/002259/part1_img1.jpg). As you can see, Rowid `87` is apparently an outlier.
+In this example, Rowid `87` is apparently an outlier.
 
 ```sh
 awk '{FS=" "; OFS=" "; print NR,$0}' hundred_balls.txt | \
@@ -144,11 +146,15 @@ where
 ;
 ```
 
-_Note: `list_neighbours` table SHOULD be created because `list_neighbours` is used multiple times._
+> #### Caution
+>
+> `list_neighbours` table SHOULD be created because `list_neighbours` is used multiple times.
 
-_Note: [`each_top_k`](https://github.com/myui/hivemall/pull/196) is supported from Hivemall v0.3.2-3 or later._
+# Parallelize Top-k computation
 
-_Note: To parallelize a top-k computation, break LEFT-hand table into piece as describe in [this page](https://github.com/myui/hivemall/wiki/Efficient-Top-k-computation-on-Apache-Hive-using-Hivemall-UDTF#parallelization-of-similarity-computation-using-with-clause)._
+> #### Info
+>
+> To parallelize a top-k computation, break LEFT-hand table into piece as describe in [this page](../misc/topk.html).
 
 ```sql
 WITH k_distance as (

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/a71bbb75/docs/gitbook/binaryclass/a9a_lr.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/binaryclass/a9a_lr.md b/docs/gitbook/binaryclass/a9a_lr.md
index 17d91c0..9bac63e 100644
--- a/docs/gitbook/binaryclass/a9a_lr.md
+++ b/docs/gitbook/binaryclass/a9a_lr.md
@@ -1,98 +1,91 @@
-<!--
-  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.
+<!--
+  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.
 -->
-        
-a9a
-===
-http://www.csie.ntu.edu.tw/~cjlin/libsvmtools/datasets/binary.html#a9a
-
-_Training with iterations is OBSOLUTE in Hivemall._  
-_Using amplifier and shuffling inputs is RECOMMENDED in Hivemall._
-
----
-
-## UDF preparation
-
-```sql
-select count(1) from a9atrain;
--- set total_steps ideally be "count(1) / #map tasks"
-set hivevar:total_steps=32561;
-
-select count(1) from a9atest;
-set hivevar:num_test_instances=16281;
-```
-
-## training
-```sql
-create table a9a_model1 
-as
-select 
- cast(feature as int) as feature,
- avg(weight) as weight
-from 
- (select 
-     logress(addBias(features),label,"-total_steps ${total_steps}") as (feature,weight)
-  from 
-     a9atrain
- ) t 
-group by feature;
-```
-_"-total_steps" option is optional for logress() function._  
-_I recommend you NOT to use options (e.g., total_steps and eta0) if you are not familiar with those options. Hivemall then uses an autonomic ETA (learning rate) estimator._
-
-## prediction
-```sql
-create or replace view a9a_predict1 
-as
-WITH a9atest_exploded as (
-select 
-  rowid,
-  label,
-  extract_feature(feature) as feature,
-  extract_weight(feature) as value
-from 
-  a9atest LATERAL VIEW explode(addBias(features)) t AS feature
-)
-select
-  t.rowid, 
-  sigmoid(sum(m.weight * t.value)) as prob,
-  CAST((case when sigmoid(sum(m.weight * t.value)) >= 0.5 then 1.0 else 0.0 end) as FLOAT) as label
-from 
-  a9atest_exploded t LEFT OUTER JOIN
-  a9a_model1 m ON (t.feature = m.feature)
-group by
-  t.rowid;
-```
-
-## evaluation
-```sql
-create or replace view a9a_submit1 as
-select 
-  t.label as actual, 
-  pd.label as predicted, 
-  pd.prob as probability
-from 
-  a9atest t JOIN a9a_predict1 pd 
-    on (t.rowid = pd.rowid);
-```
-
-```sql
-select count(1) / ${num_test_instances} from a9a_submit1 
-where actual == predicted;
-```
-> 0.8430071862907684
\ No newline at end of file
+
+<!-- toc -->
+
+# UDF preparation
+
+```sql
+select count(1) from a9atrain;
+-- set total_steps ideally be "count(1) / #map tasks"
+set hivevar:total_steps=32561;
+
+select count(1) from a9atest;
+set hivevar:num_test_instances=16281;
+```
+
+# training
+```sql
+create table a9a_model1 
+as
+select 
+ cast(feature as int) as feature,
+ avg(weight) as weight
+from 
+ (select 
+     logress(addBias(features),label,"-total_steps ${total_steps}") as (feature,weight)
+  from 
+     a9atrain
+ ) t 
+group by feature;
+```
+_"-total_steps" option is optional for logress() function._  
+_I recommend you NOT to use options (e.g., total_steps and eta0) if you are not familiar with those options. Hivemall then uses an autonomic ETA (learning rate) estimator._
+
+# prediction
+```sql
+create or replace view a9a_predict1 
+as
+WITH a9atest_exploded as (
+select 
+  rowid,
+  label,
+  extract_feature(feature) as feature,
+  extract_weight(feature) as value
+from 
+  a9atest LATERAL VIEW explode(addBias(features)) t AS feature
+)
+select
+  t.rowid, 
+  sigmoid(sum(m.weight * t.value)) as prob,
+  CAST((case when sigmoid(sum(m.weight * t.value)) >= 0.5 then 1.0 else 0.0 end) as FLOAT) as label
+from 
+  a9atest_exploded t LEFT OUTER JOIN
+  a9a_model1 m ON (t.feature = m.feature)
+group by
+  t.rowid;
+```
+
+# evaluation
+```sql
+create or replace view a9a_submit1 as
+select 
+  t.label as actual, 
+  pd.label as predicted, 
+  pd.prob as probability
+from 
+  a9atest t JOIN a9a_predict1 pd 
+    on (t.rowid = pd.rowid);
+```
+
+```sql
+select count(1) / ${num_test_instances} from a9a_submit1 
+where actual == predicted;
+```
+> 0.8430071862907684

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/a71bbb75/docs/gitbook/binaryclass/a9a_minibatch.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/binaryclass/a9a_minibatch.md b/docs/gitbook/binaryclass/a9a_minibatch.md
index eaa7a06..a79ed86 100644
--- a/docs/gitbook/binaryclass/a9a_minibatch.md
+++ b/docs/gitbook/binaryclass/a9a_minibatch.md
@@ -17,13 +17,12 @@
   under the License.
 -->
         
-This page explains how to apply [Mini-Batch Gradient Descent](https://class.coursera.org/ml-003/lecture/106) for the training of logistic regression explained in [this example](https://github.com/myui/hivemall/wiki/a9a-binary-classification-(logistic-regression)). 
-
-See [this page](https://github.com/myui/hivemall/wiki/a9a-binary-classification-(logistic-regression)) first. This content depends on it.
+This page explains how to apply [Mini-Batch Gradient Descent](https://class.coursera.org/ml-003/lecture/106) for the training of logistic regression explained in [this example](./a9a_lr.html). 
+So, refer [this page](./a9a_lr.html) first. This content depends on it.
 
 # Training
 
-Replace `a9a_model1` of [this example](https://github.com/myui/hivemall/wiki/a9a-binary-classification-(logistic-regression)).
+Replace `a9a_model1` of [this example](./a9a_lr.html).
 
 ```sql
 set hivevar:total_steps=32561;

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/a71bbb75/docs/gitbook/binaryclass/kdd2010a_dataset.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/binaryclass/kdd2010a_dataset.md b/docs/gitbook/binaryclass/kdd2010a_dataset.md
index ca221c3..7634f66 100644
--- a/docs/gitbook/binaryclass/kdd2010a_dataset.md
+++ b/docs/gitbook/binaryclass/kdd2010a_dataset.md
@@ -19,9 +19,9 @@
         
 [http://www.csie.ntu.edu.tw/~cjlin/libsvmtools/datasets/binary.html#kdd2010 (algebra)](http://www.csie.ntu.edu.tw/~cjlin/libsvmtools/datasets/binary.html#kdd2010 (algebra))
 
-* # of classes: 2
-* # of data: 8,407,752 (training) / 510,302 (testing)
-* # of features: 20,216,830 in about 2.73 GB (training) / 20,216,830 (testing) 
+* the number of classes: 2
+* the number of data: 8,407,752 (training) / 510,302 (testing)
+* the number of features: 20,216,830 in about 2.73 GB (training) / 20,216,830 (testing) 
 
 ---
 # Define training/testing tables

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/a71bbb75/docs/gitbook/binaryclass/kdd2010b_dataset.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/binaryclass/kdd2010b_dataset.md b/docs/gitbook/binaryclass/kdd2010b_dataset.md
index 41f0513..291a783 100644
--- a/docs/gitbook/binaryclass/kdd2010b_dataset.md
+++ b/docs/gitbook/binaryclass/kdd2010b_dataset.md
@@ -19,9 +19,9 @@
         
 [http://www.csie.ntu.edu.tw/~cjlin/libsvmtools/datasets/binary.html#kdd2010 (bridge to algebra)](http://www.csie.ntu.edu.tw/~cjlin/libsvmtools/datasets/binary.html#kdd2010 (bridge to algebra))
 
-* # of classes: 2
-* # of data: 19,264,097 / 748,401 (testing)
-* # of features: 29,890,095 / 29,890,095 (testing)
+* the number of classes: 2
+* the number of examples: 19,264,097 (training) / 748,401 (testing)
+* the number of features: 29,890,095 (training) / 29,890,095 (testing)
 
 ---
 # Define training/testing tables

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/a71bbb75/docs/gitbook/binaryclass/news20_scw.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/binaryclass/news20_scw.md b/docs/gitbook/binaryclass/news20_scw.md
index fa1da7f..c3f51f4 100644
--- a/docs/gitbook/binaryclass/news20_scw.md
+++ b/docs/gitbook/binaryclass/news20_scw.md
@@ -16,7 +16,7 @@
   specific language governing permissions and limitations
   under the License.
 -->
-        
+
 ## UDF preparation
 ```
 use news20;

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/a71bbb75/docs/gitbook/binaryclass/titanic_rf.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/binaryclass/titanic_rf.md b/docs/gitbook/binaryclass/titanic_rf.md
new file mode 100644
index 0000000..1a9786e
--- /dev/null
+++ b/docs/gitbook/binaryclass/titanic_rf.md
@@ -0,0 +1,318 @@
+<!--
+  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.
+-->
+
+This examples gives a basic usage of RandomForest on Hivemall using [Kaggle Titanic](https://www.kaggle.com/c/titanic) dataset.
+The example gives a baseline score without any feature engineering.
+
+<!-- toc -->
+
+# Data preparation
+
+```sql
+create database titanic;
+use titanic;
+
+drop table train;
+create external table train (
+  passengerid int, -- unique id
+  survived int, -- target label
+  pclass int,
+  name string,
+  sex string,
+  age int,
+  sibsp int, -- Number of Siblings/Spouses Aboard
+  parch int, -- Number of Parents/Children Aboard
+  ticket string,
+  fare double,
+  cabin string,
+  embarked string
+) 
+ROW FORMAT DELIMITED
+   FIELDS TERMINATED BY '|'
+   LINES TERMINATED BY '\n'
+STORED AS TEXTFILE LOCATION '/dataset/titanic/train';
+
+hadoop fs -rm /dataset/titanic/train/train.csv
+awk '{ FPAT="([^,]*)|(\"[^\"]+\")";OFS="|"; } NR >1 {$1=$1;$4=substr($4,2,length($4)-2);print $0}' train.csv | hadoop fs -put - /dataset/titanic/train/train.csv
+
+drop table test_raw;
+create external table test_raw (
+  passengerid int,
+  pclass int,
+  name string,
+  sex string,
+  age int,
+  sibsp int, -- Number of Siblings/Spouses Aboard
+  parch int, -- Number of Parents/Children Aboard
+  ticket string,
+  fare double,
+  cabin string,
+  embarked string
+)
+ROW FORMAT DELIMITED
+   FIELDS TERMINATED BY '|'
+   LINES TERMINATED BY '\n'
+STORED AS TEXTFILE LOCATION '/dataset/titanic/test_raw';
+
+hadoop fs -rm /dataset/titanic/test_raw/test.csv
+awk '{ FPAT="([^,]*)|(\"[^\"]+\")";OFS="|"; } NR >1 {$1=$1;$3=substr($3,2,length($3)-2);print $0}' test.csv | hadoop fs -put - /dataset/titanic/test_raw/test.csv
+```
+
+## Data preparation for RandomForest
+
+```sql
+set hivevar:output_row=true;
+
+drop table train_rf;
+create table train_rf
+as
+WITH train_quantified as (
+  select    
+    quantify(
+      ${output_row}, passengerid, survived, pclass, name, sex, age, sibsp, parch, ticket, fare, cabin, embarked
+    ) as (passengerid, survived, pclass, name, sex, age, sibsp, parch, ticket, fare, cabin, embarked)
+  from (
+    select * from train
+    order by passengerid asc
+  ) t
+)
+select
+  rand(31) as rnd,
+  passengerid, 
+  array(pclass, name, sex, age, sibsp, parch, ticket, fare, cabin, embarked) as features,
+  survived
+from
+  train_quantified
+;
+
+drop table test_rf;
+create table test_rf
+as
+WITH test_quantified as (
+  select 
+    quantify(
+      output_row, passengerid, pclass, name, sex, age, sibsp, parch, ticket, fare, cabin, embarked
+    ) as (passengerid, pclass, name, sex, age, sibsp, parch, ticket, fare, cabin, embarked)
+  from (
+    -- need training data to assign consistent ids to categorical variables
+    select * from (
+      select
+        1 as train_first, false as output_row, passengerid, pclass, name, sex, age, sibsp, parch, ticket, fare, cabin, embarked
+      from
+        train
+      union all
+      select
+        2 as train_first, true as output_row, passengerid, pclass, name, sex, age, sibsp, parch, ticket, fare, cabin, embarked
+      from
+        test_raw
+    ) t0
+    order by train_first asc, passengerid asc
+  ) t1
+)
+select
+  passengerid, 
+  array(pclass, name, sex, age, sibsp, parch, ticket, fare, cabin, embarked) as features
+from
+  test_quantified
+;
+```
+
+---
+
+# Training
+
+`select guess_attribute_types(pclass, name, sex, age, sibsp, parch, ticket, fare, cabin, embarked) from train limit 1;`
+> Q,C,C,Q,Q,Q,C,Q,C,C
+
+`Q` and `C` represent quantitative variable and categorical variables, respectively.
+
+*Caution:* Note that the output of `guess_attribute_types` is not perfect. Revise it by your self.
+For example, `pclass` is a categorical variable.
+
+```sql
+set hivevar:attrs=C,C,C,Q,Q,Q,C,Q,C,C;
+
+drop table model_rf;
+create table model_rf
+AS
+select
+  train_randomforest_classifier(features, survived, "-trees 500 -attrs ${attrs}") 
+    -- as (model_id, model_type, pred_model, var_importance, oob_errors, oob_tests)
+from
+  train_rf
+;
+
+select
+  array_sum(var_importance) as var_importance,
+  sum(oob_errors) / sum(oob_tests) as oob_err_rate
+from
+  model_rf;
+
+> [137.00242639169272,1194.2140119834373,328.78017188176966,628.2568660509628,200.31275032394072,160.12876797647078,1083.5987543408116,664.1234312561456,422.89449844090393,130.72019667694784]     0.18742985409652077
+```
+
+# Prediction
+
+```sql
+SET hivevar:classification=true;
+set hive.auto.convert.join=true;
+SET hive.mapjoin.optimized.hashtable=false;
+SET mapred.reduce.tasks=16;
+
+drop table predicted_rf;
+create table predicted_rf
+as
+SELECT 
+  passengerid,
+  predicted.label,
+  predicted.probability,
+  predicted.probabilities
+FROM (
+  SELECT
+    passengerid,
+    rf_ensemble(predicted) as predicted
+  FROM (
+    SELECT
+      t.passengerid, 
+      -- hivemall v0.4.1-alpha.2 or before
+      -- tree_predict(p.model, t.features, ${classification}) as predicted
+\u3000\u3000   -- hivemall v0.4.1-alpha.3 or later
+      tree_predict(p.model_id, p.model_type, p.pred_model, t.features, ${classification}) as predicted
+    FROM (
+      SELECT model_id, model_type, pred_model FROM model_rf 
+      DISTRIBUTE BY rand(1)
+    ) p
+    LEFT OUTER JOIN test_rf t
+  ) t1
+  group by
+    passengerid
+) t2
+;
+```
+
+# Kaggle submission
+
+```sql
+drop table predicted_rf_submit;
+create table predicted_rf_submit
+  ROW FORMAT DELIMITED 
+    FIELDS TERMINATED BY ","
+    LINES TERMINATED BY "\n"
+  STORED AS TEXTFILE
+as
+SELECT passengerid, label as survived
+FROM predicted_rf
+ORDER BY passengerid ASC;
+```
+
+```sh
+hadoop fs -getmerge /user/hive/warehouse/titanic.db/predicted_rf_submit predicted_rf_submit.csv
+
+sed -i -e "1i PassengerId,Survived" predicted_rf_submit.csv
+```
+
+Accuracy would gives `0.76555` for a Kaggle submission.
+
+---
+
+# Test by dividing training dataset
+
+```sql
+drop table train_rf_07;
+create table train_rf_07 
+as
+select * from train_rf 
+where rnd < 0.7;
+
+drop table test_rf_03;
+create table test_rf_03
+as
+select * from train_rf
+where rnd >= 0.7;
+
+drop table model_rf_07;
+create table model_rf_07
+AS
+select
+  train_randomforest_classifier(features, survived, "-trees 500 -attrs ${attrs}") 
+from
+  train_rf_07;
+
+select
+  array_sum(var_importance) as var_importance,
+  sum(oob_errors) / sum(oob_tests) as oob_err_rate
+from
+  model_rf_07;
+> [116.12055542977338,960.8569891444097,291.08765260103837,469.74671636586226,163.721292772701,120.784769882858,847.9769298113661,554.4617571355476,346.3500941757221,97.42593940113392]    0.1838351822503962
+
+SET hivevar:classification=true;
+SET hive.mapjoin.optimized.hashtable=false;
+SET mapred.reduce.tasks=16;
+
+drop table predicted_rf_03;
+create table predicted_rf_03
+as
+SELECT 
+  passengerid,
+  predicted.label,
+  predicted.probability,
+  predicted.probabilities
+FROM (
+  SELECT
+    passengerid,
+    rf_ensemble(predicted) as predicted
+  FROM (
+    SELECT
+      t.passengerid, 
+      -- hivemall v0.4.1-alpha.2 or before
+      -- tree_predict(p.model, t.features, ${classification}) as predicted
+      -- hivemall v0.4.1-alpha.3 or later
+      tree_predict(p.model_id, p.model_type, p.pred_model, t.features, ${classification}) as predicted
+    FROM (
+      SELECT model_id, model_type, pred_model FROM model_rf_07
+      DISTRIBUTE BY rand(1)
+    ) p
+    LEFT OUTER JOIN test_rf_03 t
+  ) t1
+  group by
+    passengerid
+) t2
+;
+
+create or replace view rf_submit_03 as
+select 
+  t.survived as actual, 
+  p.label as predicted,
+  p.probabilities
+from 
+  test_rf_03 t 
+  JOIN predicted_rf_03 p on (t.passengerid = p.passengerid)
+;
+
+select count(1) from test_rf_03;
+> 260
+
+set hivevar:testcnt=260;
+
+select count(1)/${testcnt} as accuracy 
+from rf_submit_03 
+where actual = predicted;
+
+> 0.8
+```

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/a71bbb75/docs/gitbook/binaryclass/webspam_scw.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/binaryclass/webspam_scw.md b/docs/gitbook/binaryclass/webspam_scw.md
index cadd0ab..067e8f2 100644
--- a/docs/gitbook/binaryclass/webspam_scw.md
+++ b/docs/gitbook/binaryclass/webspam_scw.md
@@ -152,4 +152,4 @@ from
 select count(1)/70000 from webspam_scw_submit1 
 where actual = predicted;
 ```
-> Prediction accuracy: 0.9778714285714286
\ No newline at end of file
+> Prediction accuracy: 0.9778714285714286

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/a71bbb75/docs/gitbook/eval/lr_datagen.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/eval/lr_datagen.md b/docs/gitbook/eval/lr_datagen.md
index 8fa5239..c0cbce0 100644
--- a/docs/gitbook/eval/lr_datagen.md
+++ b/docs/gitbook/eval/lr_datagen.md
@@ -17,7 +17,7 @@
   under the License.
 -->
         
-_Note this feature is supported on hivemall v0.2-alpha3 or later._
+<!-- toc -->
 
 # create a dual table
 
@@ -33,10 +33,10 @@ INSERT INTO TABLE dual SELECT count(*)+1 FROM dual;
 ```sql
 create table regression_data1
 as
-select lr_datagen("-n_examples 10k -n_features 10 -seed 100") as (label,features)
+select lr_datagen('-n_examples 10k -n_features 10 -seed 100') as (label,features)
 from dual;
 ```
-Find the details of the option in [LogisticRegressionDataGeneratorUDTF.java](https://github.com/myui/hivemall/blob/master/core/src/main/java/hivemall/dataset/LogisticRegressionDataGeneratorUDTF.java#L69).
+Find the details of the option, run `lr_datagen('-help')`.
 
 You can generate a sparse dataset as well as a dense dataset. By the default, a sparse dataset is generated.
 ```sql

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/a71bbb75/docs/gitbook/eval/stat_eval.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/eval/stat_eval.md b/docs/gitbook/eval/stat_eval.md
index 6b0af8e..149adf8 100644
--- a/docs/gitbook/eval/stat_eval.md
+++ b/docs/gitbook/eval/stat_eval.md
@@ -17,7 +17,9 @@
   under the License.
 -->
         
-Using the [E2006 tfidf regression example](https://github.com/myui/hivemall/wiki/E2006-tfidf-regression-evaluation-(PA,-AROW)), we explain how to evaluate the prediction model on Hive.
+Using the [E2006 tfidf regression example](../regression/e2006_arow.html), we explain how to evaluate the prediction model on Hive.
+
+<!-- toc -->
 
 # Scoring by evaluation metrics
 
@@ -69,7 +71,7 @@ from t;
 ```
 > 1.9610366706408238	1.9610366706408238
 
---
-**References**
+# References
+
 * R2 http://en.wikipedia.org/wiki/Coefficient_of_determination
-* Evaluation Metrics https://www.kaggle.com/wiki/Metrics
\ No newline at end of file
+* Evaluation Metrics https://www.kaggle.com/wiki/Metrics

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/a71bbb75/docs/gitbook/ft_engineering/hashing.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/ft_engineering/hashing.md b/docs/gitbook/ft_engineering/hashing.md
index daf4a23..f467002 100644
--- a/docs/gitbook/ft_engineering/hashing.md
+++ b/docs/gitbook/ft_engineering/hashing.md
@@ -17,10 +17,10 @@
   under the License.
 -->
         
-Hivemall supports [Feature Hashing](https://github.com/myui/hivemall/wiki/Feature-hashing) (a.k.a. hashing trick) through `feature_hashing` and `mhash` functions. 
+Hivemall supports [Feature Hashing](https://en.wikipedia.org/wiki/Feature_hashing) (a.k.a. hashing trick) through `feature_hashing` and `mhash` functions. 
 Find the differences in the following examples.
 
-_Note: `feature_hashing` UDF is supported since Hivemall `v0.4.2-rc.1`._
+<!-- toc -->
 
 ## `feature_hashing` function
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/a71bbb75/docs/gitbook/getting_started/input-format.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/getting_started/input-format.md b/docs/gitbook/getting_started/input-format.md
index 698c095..59e6a5f 100644
--- a/docs/gitbook/getting_started/input-format.md
+++ b/docs/gitbook/getting_started/input-format.md
@@ -24,14 +24,14 @@ Here, we use [EBNF](http://en.wikipedia.org/wiki/Extended_Backus%E2%80%93Naur_Fo
 
 # Input Format for Classification 
 
-The classifiers of Hivemall takes 2 (or 3) arguments: *features*, *label*, and *options* (a.k.a. [hyperparameters](http://en.wikipedia.org/wiki/Hyperparameter)). The first two arguments of training functions (e.g., [logress](https://github.com/myui/hivemall/wiki/a9a-binary-classification-(logistic-regression)) and [train_scw](https://github.com/myui/hivemall/wiki/news20-binary-classification-%232-(CW,-AROW,-SCW))) represents training examples. 
+The classifiers of Hivemall takes 2 (or 3) arguments: *features*, *label*, and *options* (a.k.a. [hyperparameters](http://en.wikipedia.org/wiki/Hyperparameter)). The first two arguments of training functions represents training examples. 
 
 In Statistics, *features* and *label* are called [Explanatory variable and Response Variable](http://www.oswego.edu/~srp/stats/variable_types.htm), respectively.
 
 # Features format (for classification and regression)
 
 The format of *features* is common between (binary and multi-class) classification and regression.
-Hivemall accepts ARRAY&lt;INT|BIGINT|TEXT> for the type of *features* column.
+Hivemall accepts `ARRAY&lt;INT|BIGINT|TEXT>` for the type of *features* column.
 
 Hivemall uses a *sparse* data format (cf. [Compressed Row Storage](http://netlib.org/linalg/html_templates/node91.html)) which is similar to [LIBSVM](http://stackoverflow.com/questions/12112558/read-write-data-in-libsvm-format) and [Vowpal Wabbit](https://github.com/JohnLangford/vowpal_wabbit/wiki/Input-format).
 
@@ -52,7 +52,7 @@ Here is an instance of a features.
 10:3.4  123:0.5  34567:0.231
 ```
 
-*Note:* As mentioned later, *index* "0" is reserved for a [Bias/Dummy variable](https://github.com/myui/hivemall/wiki/Using-explicit-addBias()-for-a-better-prediction).
+*Note:* As mentioned later, *index* "0" is reserved for a [Bias/Dummy variable](../tips/addbias.html).
 
 In addition to numbers, you can use a TEXT value for an index. For example, you can use array("height:1.5", "length:2.0") for the features.
 ```
@@ -80,15 +80,15 @@ Note 1.0 is used for the weight when omitting *weight*.
 
 Note that "0" is reserved for a Bias variable (called dummy variable in Statistics). 
 
-The [addBias](https://github.com/myui/hivemall/wiki/Using-explicit-addBias()-for-a-better-prediction) function is Hivemall appends "0:1.0" as an element of array in *features*.
+The [addBias](../tips/addbias.html) function is Hivemall appends "0:1.0" as an element of array in *features*.
 
 ## Feature hashing
 
-Hivemall supports [feature hashing/hashing trick](http://en.wikipedia.org/wiki/Feature_hashing) through [mhash function](https://github.com/myui/hivemall/wiki/KDDCup-2012-track-2-CTR-prediction-dataset#converting-feature-representation-by-feature-hashing).
+Hivemall supports [feature hashing/hashing trick](http://en.wikipedia.org/wiki/Feature_hashing) through [mhash function](../ft_engineering/hashing.html#mhash-function).
 
 The mhash function takes a feature (i.e., *index*) of TEXT format and generates a hash number of a range from 1 to 2^24 (=16777216) by the default setting.
 
-Feature hashing is useful where the dimension of feature vector (i.e., the number of elements in *features*) is so large. Consider applying [mhash function]((https://github.com/myui/hivemall/wiki/KDDCup-2012-track-2-CTR-prediction-dataset#converting-feature-representation-by-feature-hashing)) when a prediction model does not fit in memory and OutOfMemory exception happens.
+Feature hashing is useful where the dimension of feature vector (i.e., the number of elements in *features*) is so large. Consider applying [mhash function]((../ft_engineering/hashing.html#mhash-function)) when a prediction model does not fit in memory and OutOfMemory exception happens.
 
 In general, you don't need to use mhash when the dimension of feature vector is less than 16777216.
 If feature *index* is very long TEXT (e.g., "xxxxxxx-yyyyyy-weight:55.3") and uses huge memory spaces, consider using mhash as follows:
@@ -103,7 +103,7 @@ feature(mhash(extract_feature("xxxxxxx-yyyyyy-weight:55.3")), extract_weight("xx
 
 ## Feature Normalization
 
-Feature (weight) normalization is important in machine learning. Please refer [https://github.com/myui/hivemall/wiki/Feature-scaling](https://github.com/myui/hivemall/wiki/Feature-scaling) for detail.
+Feature (weight) normalization is important in machine learning. Please refer [this article](../ft_engineering/scaling.html) for detail.
 
 ***
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/a71bbb75/docs/gitbook/getting_started/permanent-functions.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/getting_started/permanent-functions.md b/docs/gitbook/getting_started/permanent-functions.md
index 75156fe..7afc780 100644
--- a/docs/gitbook/getting_started/permanent-functions.md
+++ b/docs/gitbook/getting_started/permanent-functions.md
@@ -21,8 +21,6 @@ Hive v0.13 or later supports [permanent functions](https://cwiki.apache.org/conf
 
 Permanent functions are useful when you are using Hive through Hiveserver or to avoid hivemall installation for each session.
 
-_Note: This feature is supported since hivemall-0.3 beta 3 or later._
-
 <!-- toc -->
 
 # Put hivemall jar to HDFS
@@ -58,4 +56,5 @@ show functions "hivemall.*";
 ```
 
 > #### Caution
-You need to specify "hivemall." prefix to call hivemall UDFs in your queries if UDFs are loaded into non-default scheme, in this case _hivemall_.
+>
+> You need to specify "hivemall." prefix to call hivemall UDFs in your queries if UDFs are loaded into non-default scheme, in this case _hivemall_.

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/a71bbb75/docs/gitbook/misc/generic_funcs.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/misc/generic_funcs.md b/docs/gitbook/misc/generic_funcs.md
index 9749dae..b3a0421 100644
--- a/docs/gitbook/misc/generic_funcs.md
+++ b/docs/gitbook/misc/generic_funcs.md
@@ -19,61 +19,63 @@
         
 This page describes a list of useful Hivemall generic functions.
 
+<!-- toc -->
+
 # Array functions
 
 ## Array UDFs
 
 - `array_concat(array<ANY> x1, array<ANY> x2, ..)` - Returns a concatenated array
 
-```sql
-select array_concat(array(1),array(2,3));
-> [1,2,3]
-```
+    ```sql
+    select array_concat(array(1),array(2,3));
+    > [1,2,3]
+    ```
 
 - `array_intersect(array<ANY> x1, array<ANY> x2, ..)` - Returns an intersect of given arrays
 
-```sql
-select array_intersect(array(1,3,4),array(2,3,4),array(3,5));
-> [3]
-```
+    ```sql
+    select array_intersect(array(1,3,4),array(2,3,4),array(3,5));
+    > [3]
+    ```
 
 - `array_remove(array<int|text> original, int|text|array<int> target)` - Returns an array that the target is removed from the original array
 
-```sql
-select array_remove(array(1,null,3),array(null));
-> [3]
-
-select array_remove(array("aaa","bbb"),"bbb");
-> ["aaa"]
-```
+    ```sql
+    select array_remove(array(1,null,3),array(null));
+    > [3]
+    
+    select array_remove(array("aaa","bbb"),"bbb");
+    > ["aaa"]
+    ```
 
-- `sort_and_uniq_array(array<int>)` - Takes an array of type int and returns a sorted array in a natural order with duplicate elements eliminated
+- `sort_and_uniq_array(array<int>)` - Takes an array of type INT and returns a sorted array in a natural order with duplicate elements eliminated
 
-```sql
-select sort_and_uniq_array(array(3,1,1,-2,10));
-> [-2,1,3,10]
-```
+    ```sql
+    select sort_and_uniq_array(array(3,1,1,-2,10));
+    > [-2,1,3,10]
+    ```
 
 - `subarray_endwith(array<int|text> original, int|text key)` - Returns an array that ends with the specified key
-
-```sql
-select subarray_endwith(array(1,2,3,4), 3);
-> [1,2,3]
-```
+    
+    ```sql
+    select subarray_endwith(array(1,2,3,4), 3);
+    > [1,2,3]
+    ```
 
 - `subarray_startwith(array<int|text> original, int|text key)` - Returns an array that starts with the specified key
 
-```sql
-select subarray_startwith(array(1,2,3,4), 2);
-> [2,3,4]
-```
+    ```sql
+    select subarray_startwith(array(1,2,3,4), 2);
+    > [2,3,4]
+    ```
 
-- `subarray(array<int> orignal, int fromIndex, int toIndex)` - Returns a slice of the original array between the inclusive fromIndex and the exclusive toIndex
+- `subarray(array<int> orignal, int fromIndex, int toIndex)` - Returns a slice of the original array between the inclusive `fromIndex` and the exclusive `toIndex`
 
-```sql
-select subarray(array(1,2,3,4,5,6), 2,4);
-> [3,4]
-```
+    ```sql
+    select subarray(array(1,2,3,4,5,6), 2,4);
+    > [3,4]
+    ```
 
 ## Array UDAFs
 
@@ -87,47 +89,45 @@ select subarray(array(1,2,3,4,5,6), 2,4);
 
 - `to_bits(int[] indexes)` - Returns an bitset representation if the given indexes in long[]
 
-```sql
-select to_bits(array(1,2,3,128));
->[14,-9223372036854775808]
-```
+    ```sql
+    select to_bits(array(1,2,3,128));
+    >[14,-9223372036854775808]
+    ```
 
 - `unbits(long[] bitset)` - Returns an long array of the give bitset representation
 
-```sql
-select unbits(to_bits(array(1,4,2,3)));
-> [1,2,3,4]
-```
+    ```sql
+    select unbits(to_bits(array(1,4,2,3)));
+    > [1,2,3,4]
+    ```
 
 - `bits_or(array<long> b1, array<long> b2, ..)` - Returns a logical OR given bitsets
 
-```sql
-select unbits(bits_or(to_bits(array(1,4)),to_bits(array(2,3))));
-> [1,2,3,4]
-```
+    ```sql
+    select unbits(bits_or(to_bits(array(1,4)),to_bits(array(2,3))));
+    > [1,2,3,4]
+    ```
 
 ## Bitset UDAF
 
 - `bits_collect(int|long x)` - Returns a bitset in array<long>
 
-
 # Compression functions
 
-- `deflate(TEXT data [, const int compressionLevel])` - Returns a compressed BINARY obeject by using Deflater.
+- `deflate(TEXT data [, const int compressionLevel])` - Returns a compressed BINARY object by using Deflater.
 The compression level must be in range [-1,9]
 
-```sql
-select base91(deflate('aaaaaaaaaaaaaaaabbbbccc'));
-> AA+=kaIM|WTt!+wbGAA
-```
+    ```sql
+    select base91(deflate('aaaaaaaaaaaaaaaabbbbccc'));
+    > AA+=kaIM|WTt!+wbGAA
+    ```
 
 - `inflate(BINARY compressedData)` - Returns a decompressed STRING by using Inflater
 
-
-```sql
-select inflate(unbase91(base91(deflate('aaaaaaaaaaaaaaaabbbbccc'))));
-> aaaaaaaaaaaaaaaabbbbccc
-```
+    ```sql
+    select inflate(unbase91(base91(deflate('aaaaaaaaaaaaaaaabbbbccc'))));
+    > aaaaaaaaaaaaaaaabbbbccc
+    ```
 
 # Map functions
 
@@ -152,33 +152,33 @@ select inflate(unbase91(base91(deflate('aaaaaaaaaaaaaaaabbbbccc'))));
 
 # Math functions
 
-- `sigmoid(x)` - Returns 1.0 / (1.0 + exp(-x))
+- `sigmoid(x)` - Returns `1.0 / (1.0 + exp(-x))`
 
 # Text processing functions
 
 - `base91(binary)` - Convert the argument from binary to a BASE91 string
 
-```sql
-select base91(deflate('aaaaaaaaaaaaaaaabbbbccc'));
-> AA+=kaIM|WTt!+wbGAA
-```
+    ```sql
+    select base91(deflate('aaaaaaaaaaaaaaaabbbbccc'));
+    > AA+=kaIM|WTt!+wbGAA
+    ```
 
 - `unbase91(string)` - Convert a BASE91 string to a binary
 
-```sql
-select inflate(unbase91(base91(deflate('aaaaaaaaaaaaaaaabbbbccc'))));
-> aaaaaaaaaaaaaaaabbbbccc
-```
+    ```sql
+    select inflate(unbase91(base91(deflate('aaaaaaaaaaaaaaaabbbbccc'))));
+    > aaaaaaaaaaaaaaaabbbbccc
+    ```
 
 - `normalize_unicode(string str [, string form])` - Transforms `str` with the specified normalization form. The `form` takes one of NFC (default), NFD, NFKC, or NFKD
 
-```sql
-select normalize_unicode('\uff8a\uff9d\uff76\uff78\uff76\uff85','NFKC');
-> \u30cf\u30f3\u30ab\u30af\u30ab\u30ca
-
-select normalize_unicode('\u3231\u3327\u3326\u2162','NFKC');
-> (\u682a)\u30c8\u30f3\u30c9\u30ebIII
-```
+    ```sql
+    select normalize_unicode('\uff8a\uff9d\uff76\uff78\uff76\uff85','NFKC');
+    > \u30cf\u30f3\u30ab\u30af\u30ab\u30ca
+    
+    select normalize_unicode('\u3231\u3327\u3326\u2162','NFKC');
+    > (\u682a)\u30c8\u30f3\u30c9\u30ebIII
+    ```
 
 - `split_words(string query [, string regex])` - Returns an array<text> containing splitted strings
 
@@ -186,44 +186,37 @@ select normalize_unicode('\u3231\u3327\u3326\u2162','NFKC');
 
 - `tokenize(string englishText [, boolean toLowerCase])` - Returns words in array<string>
 
-- `tokenize_ja(String line [, const string mode = "normal", const list<string> stopWords, const list<string> stopTags])` - returns tokenized strings in array<string>
-
-```sql
-select tokenize_ja("kuromoji\u3092\u4f7f\u3063\u305f\u5206\u304b\u3061\u66f8\u304d\u306e\u30c6\u30b9\u30c8\u3067\u3059\u3002\u7b2c\u4e8c\u5f15\u6570\u306b\u306fnormal/search/extended\u3092\u6307\u5b9a\u3067\u304d\u307e\u3059\u3002\u30c7\u30d5\u30a9\u30eb\u30c8\u3067\u306fnormal\u30e2\u30fc\u30c9\u3067\u3059\u3002");
+- `tokenize_ja(String line [, const string mode = "normal", const list<string> stopWords, const list<string> stopTags])` - returns tokenized strings in array<string>. Refer [this article](../misc/tokenizer.html) for detail.
 
-> ["kuromoji","\u4f7f\u3046","\u5206\u304b\u3061\u66f8\u304d","\u30c6\u30b9\u30c8","\u7b2c","\u4e8c","\u5f15\u6570","normal","search","extended","\u6307\u5b9a","\u30c7\u30d5\u30a9\u30eb\u30c8","normal"," \u30e2\u30fc\u30c9"]
-```
-
-https://github.com/myui/hivemall/wiki/Tokenizer
+    ```sql
+    select tokenize_ja("kuromoji\u3092\u4f7f\u3063\u305f\u5206\u304b\u3061\u66f8\u304d\u306e\u30c6\u30b9\u30c8\u3067\u3059\u3002\u7b2c\u4e8c\u5f15\u6570\u306b\u306fnormal/search/extended\u3092\u6307\u5b9a\u3067\u304d\u307e\u3059\u3002\u30c7\u30d5\u30a9\u30eb\u30c8\u3067\u306fnormal\u30e2\u30fc\u30c9\u3067\u3059\u3002");
+    
+    > ["kuromoji","\u4f7f\u3046","\u5206\u304b\u3061\u66f8\u304d","\u30c6\u30b9\u30c8","\u7b2c","\u4e8c","\u5f15\u6570","normal","search","extended","\u6307\u5b9a","\u30c7\u30d5\u30a9\u30eb\u30c8","normal"," \u30e2\u30fc\u30c9"]
+    ```
 
 # Other functions
 
 - `convert_label(const int|const float)` - Convert from -1|1 to 0.0f|1.0f, or from 0.0f|1.0f to -1|1
 
-- `each_top_k(int K, Object group, double cmpKey, *)` - Returns top-K values (or tail-K values when k is less than 0)
-
-https://github.com/myui/hivemall/wiki/Efficient-Top-k-computation-on-Apache-Hive-using-Hivemall-UDTF
+- `each_top_k(int K, Object group, double cmpKey, *)` - Returns top-K values (or tail-K values when k is less than 0). Refer [this article](../misc/topk.html) for detail.
 
 - `generate_series(const int|bigint start, const int|bigint end)` - Generate a series of values, from start to end
 
-```sql
-WITH dual as (
-  select 1
-)
-select generate_series(1,9)
-from dual;
-
-1
-2
-3
-4
-5
-6
-7
-8
-9
-```
-
-A similar function to PostgreSQL's `generate_serics`.
-http://www.postgresql.org/docs/current/static/functions-srf.html
-- `x_rank(KEY)` - Generates a pseudo sequence number starting from 1 for each key
\ No newline at end of file
+    ```sql
+    select generate_series(1,9);
+    
+    1
+    2
+    3
+    4
+    5
+    6
+    7
+    8
+    9
+    ```
+
+    A similar function to PostgreSQL's `generate_serics`.
+    http://www.postgresql.org/docs/current/static/functions-srf.html
+
+- `x_rank(KEY)` - Generates a pseudo sequence number starting from 1 for each key

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/a71bbb75/docs/gitbook/misc/topk.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/misc/topk.md b/docs/gitbook/misc/topk.md
index d6e7b93..6a80514 100644
--- a/docs/gitbook/misc/topk.md
+++ b/docs/gitbook/misc/topk.md
@@ -23,7 +23,10 @@ This function is particularly useful for applying a similarity/distance function
 
 `each_top_k` is very fast when compared to other methods running top-k queries (e.g., [`rank/distribute by`](https://ragrawal.wordpress.com/2011/11/18/extract-top-n-records-in-each-group-in-hadoophive/)) in Hive.
 
-## Caution
+<!-- toc -->
+
+# Caution
+
 * `each_top_k` is supported from Hivemall v0.3.2-3 or later.
 * This UDTF assumes that input records are sorted by `group`. Use `DISTRIBUTE BY group SORT BY group` to ensure that. Or, you can use `LEFT OUTER JOIN` for certain cases.
 * It takes variable lengths arguments in `argN`. 
@@ -32,7 +35,9 @@ This function is particularly useful for applying a similarity/distance function
 * If k is less than 0, reverse order is used and `tail-K` records are returned for each `group`.
 * Note that this function returns [a pseudo ranking](http://www.michaelpollmeier.com/selecting-top-k-items-from-a-list-efficiently-in-java-groovy/) for top-k. It always returns `at-most K` records for each group. The ranking scheme is similar to `dense_rank` but slightly different in certain cases.
 
-# Efficient Top-k Query Processing using `each_top_k`
+# Usage
+
+## Efficient Top-k Query Processing using `each_top_k`
 
 Efficient processing of Top-k queries is a crucial requirement in many interactive environments that involve massive amounts of data. 
 Our Hive extension `each_top_k` helps running Top-k processing efficiently.
@@ -87,7 +92,8 @@ FROM (
 ```
 
 > #### Note
-`CLUSTER BY x` is a synonym of `DISTRIBUTE BY x CLASS SORT BY x` and required when using `each_top_k`.
+>
+> `CLUSTER BY x` is a synonym of `DISTRIBUTE BY x CLASS SORT BY x` and required when using `each_top_k`.
 
 The function signature of `each_top_k` is `each_top_k(int k, ANY group, double value, arg1, arg2, ..., argN)` and it returns a relation `(int rank, double value, arg1, arg2, .., argN)`.
 
@@ -99,9 +105,8 @@ If `k` is less than 0, reverse order is used and tail-K records are returned for
 The ranking semantics of `each_top_k` follows SQL's `dense_rank` and then limits results by `k`. 
 
 > #### Caution
-`each_top_k` is benefical where the number of grouping keys are large. If the number of grouping keys are not so large (e.g., less than 100), consider using `rank() over` instead.
-
-# Usage
+>
+> `each_top_k` is benefical where the number of grouping keys are large. If the number of grouping keys are not so large (e.g., less than 100), consider using `rank() over` instead.
 
 ## top-k clicks 
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/a71bbb75/docs/gitbook/multiclass/iris_dataset.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/multiclass/iris_dataset.md b/docs/gitbook/multiclass/iris_dataset.md
index 38a6831..e67737e 100644
--- a/docs/gitbook/multiclass/iris_dataset.md
+++ b/docs/gitbook/multiclass/iris_dataset.md
@@ -113,7 +113,7 @@ select * from iris_scaled limit 3;
 > 3       Iris-setosa     ["1:0.11111101","2:0.5","3:0.05084745","4:0.041666664","0:1.0"]
 ```
 
-_[LibSVM web page](http://www.csie.ntu.edu.tw/~cjlin/libsvmtools/datasets/multiclass.html#iris) provides a normalized (using [ZScore](https://github.com/myui/hivemall/wiki/Feature-scaling)) version of Iris dataset._
+_[LibSVM web page](http://www.csie.ntu.edu.tw/~cjlin/libsvmtools/datasets/multiclass.html#iris) provides a normalized (using [ZScore](../ft_engineering/scaling.html#feature-scaling-by-zscore)) version of Iris dataset._
 
 # Create training/test data
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/a71bbb75/docs/gitbook/multiclass/iris_randomforest.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/multiclass/iris_randomforest.md b/docs/gitbook/multiclass/iris_randomforest.md
index fd85471..4b0750c 100644
--- a/docs/gitbook/multiclass/iris_randomforest.md
+++ b/docs/gitbook/multiclass/iris_randomforest.md
@@ -16,8 +16,6 @@
   specific language governing permissions and limitations
   under the License.
 -->
-        
-*NOTE: RandomForest is being supported from Hivemall v0.4 or later.*
 
 # Dataset
 
@@ -323,4 +321,4 @@ WHERE
   actual = predicted
 ;
 ```
-> 0.9533333333333334
\ No newline at end of file
+> 0.9533333333333334

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/a71bbb75/docs/gitbook/multiclass/iris_scw.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/multiclass/iris_scw.md b/docs/gitbook/multiclass/iris_scw.md
index fd85471..79cdaf4 100644
--- a/docs/gitbook/multiclass/iris_scw.md
+++ b/docs/gitbook/multiclass/iris_scw.md
@@ -323,4 +323,4 @@ WHERE
   actual = predicted
 ;
 ```
-> 0.9533333333333334
\ No newline at end of file
+> 0.9533333333333334

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/a71bbb75/docs/gitbook/multiclass/news20_scw.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/multiclass/news20_scw.md b/docs/gitbook/multiclass/news20_scw.md
index f6f21af..24e0fad 100644
--- a/docs/gitbook/multiclass/news20_scw.md
+++ b/docs/gitbook/multiclass/news20_scw.md
@@ -335,4 +335,4 @@ where actual == predicted;
 drop table news20mc_scw2_model1;
 drop table news20mc_scw2_predict1;
 drop view news20mc_scw2_submit1;
-```
\ No newline at end of file
+```

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/a71bbb75/docs/gitbook/recommend/item_based_cf.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/recommend/item_based_cf.md b/docs/gitbook/recommend/item_based_cf.md
index 2eb7890..a674f70 100644
--- a/docs/gitbook/recommend/item_based_cf.md
+++ b/docs/gitbook/recommend/item_based_cf.md
@@ -90,7 +90,7 @@ group by
 
 **Caution:** _Item-Item cooccurrence matrix is a symmetric matrix that has the number of total occurrence for each diagonal element . If the size of items are `k`, then the size of expected matrix is `k * (k - 1) / 2`, usually a very large one._
 
-_Better to use [2.2.2.](https://github.com/myui/hivemall/wiki/Item-based-Collaborative-Filtering#limiting-size-of-elements-in-cooccurrence_upper_triangular) instead of [2.2.1.](https://github.com/myui/hivemall/wiki/Item-based-Collaborative-Filtering#221-create-cooccurrence-table-directly) for creating a `cooccurrence` table where dataset is large._
+_Better to use [2.2.2.](#222-create-cooccurrence-table-from-upper-triangular-matrix-of-cooccurrence) instead of [2.2.1.](#221-create-cooccurrence-table-directly) for creating a `cooccurrence` table where dataset is large._
 
 ### 2.2.1. Create cooccurrence table directly
 
@@ -257,7 +257,7 @@ GROUP BY
 Item-Item similarity computation is known to be computation complexity `O(n^2)` where `n` is the number of items.
 Depending on your cluster size and your dataset, the optimal solution differs.
 
-**Note:** _Better to use [3.1.1.](https://github.com/myui/hivemall/wiki/Item-based-Collaborative-Filtering#311-similarity-computation-using-the-symmetric-property-of-item-similarity-matrix) scheme where dataset is large._
+**Note:** _Better to use [3.1.1.](#311-similarity-computation-using-the-symmetric-property-of-item-similarity-matrix) scheme where dataset is large._
 
 ### 3.1. Shuffle heavy similarity computation
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/a71bbb75/docs/gitbook/recommend/movielens_fm.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/recommend/movielens_fm.md b/docs/gitbook/recommend/movielens_fm.md
index eac8013..ad59324 100644
--- a/docs/gitbook/recommend/movielens_fm.md
+++ b/docs/gitbook/recommend/movielens_fm.md
@@ -21,8 +21,7 @@ _Caution: Factorization Machine is supported from Hivemall v0.4 or later._
 
 # Data preparation
 
-First of all, please create `ratings` table described in the following page: 
-https://github.com/myui/hivemall/wiki/MovieLens-Dataset
+First of all, please create `ratings` table described in [this article](../recommend/movielens_dataset.html).
 
 ```sql
 use movielens;
@@ -190,7 +189,7 @@ usage: train_fm(array<string> x, double y [, const string options]) -
 
 ```sql
 -- workaround for a bug 
--- https://github.com/myui/hivemall/wiki/Map-side-Join-causes-ClassCastException-on-Tez:-LazyBinaryArray-cannot-be-cast-to-%5BLjava.lang.Object;
+-- https://issues.apache.org/jira/browse/HIVE-11051
 set hive.mapjoin.optimized.hashtable=false;
 
 drop table fm_predict;
@@ -222,7 +221,7 @@ from
 # Fast Factorization Machines Training using Int Features
 
 Training of Factorization Machines (FM) can be done more efficietly, in term of speed, by using INT features.
-In this section, we show how to run FM training by using int features, more specifically by using [feature hashing](https://github.com/myui/hivemall/wiki/Feature-hashing).
+In this section, we show how to run FM training by using int features, more specifically by using [feature hashing](../ft_engineering/hashing.html).
 
 ```sql
 set hivevar:factor=10;

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/a71bbb75/docs/gitbook/recommend/movielens_mf.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/recommend/movielens_mf.md b/docs/gitbook/recommend/movielens_mf.md
index f275df8..ca38fec 100644
--- a/docs/gitbook/recommend/movielens_mf.md
+++ b/docs/gitbook/recommend/movielens_mf.md
@@ -17,9 +17,9 @@
   under the License.
 -->
         
-This page explains how to run matrix factorization on [MovieLens 1M dataset](https://github.com/myui/hivemall/wiki/MovieLens-Dataset).
+This page explains how to run matrix factorization on [MovieLens 1M dataset](../recommend/movielens_dataset.html).
 
-*Caution:* Matrix factorization is supported in Hivemall v0.3 or later.
+<!-- toc -->
 
 ## Calculate the mean rating in the training dataset
 ```sql
@@ -38,9 +38,8 @@ set hivevar:factor=10;
 -- maximum number of training iterations
 set hivevar:iters=50;
 ```
-See [this article](https://github.com/myui/hivemall/wiki/List-of-parameters-of-Matrix-Factorization) or [OnlineMatrixFactorizationUDTF#getOption()](https://github.com/myui/hivemall/blob/master/src/main/java/hivemall/mf/OnlineMatrixFactorizationUDTF.java#L123) to get the details of options.
 
-Note that there are no need to set an exact value for $mu. It actually works without setting $mu but recommended to set one for getting a better prediction.
+Note that there are no need to set an exact value for `$mu`. It actually works without setting `$mu` but recommended to set one for getting a better prediction.
 
 _Due to [a bug](https://issues.apache.org/jira/browse/HIVE-8396) in Hive, do not issue comments in CLI._
 
@@ -56,13 +55,17 @@ select
   avg(m_bias) as Bi
 from (
   select 
-    train_mf_sgd(userid, movieid, rating, "-factor ${factor} -mu ${mu} -iter ${iters}") as (idx, u_rank, m_rank, u_bias, m_bias)
+    train_mf_sgd(userid, movieid, rating, '-factor ${factor} -mu ${mu} -iter ${iters}') as (idx, u_rank, m_rank, u_bias, m_bias)
   from 
     training
 ) t
 group by idx;
 ```
-Note: Hivemall also provides *train_mf_adagrad* for training using AdaGrad.
+
+> #### Note
+>
+> Hivemall also provides *train_mf_adagrad* for training using AdaGrad. 
+> `-help` option shows a complete list of hyperparameters.
 
 # Predict
 
@@ -109,9 +112,10 @@ from (
   ON (t2.movieid = p2.idx)
 ) t;
 ```
-> 0.6728969407733578 (MAE) 
 
-> 0.8584162122694449 (RMSE)
+| MAE | RMSE |
+|:---:|:----:|
+| 0.6728969407733578 | 0.8584162122694449 |
 
 # Item Recommendation
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/a71bbb75/docs/gitbook/recommend/news20_knn.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/recommend/news20_knn.md b/docs/gitbook/recommend/news20_knn.md
index 1e0ae97..fca9db5 100644
--- a/docs/gitbook/recommend/news20_knn.md
+++ b/docs/gitbook/recommend/news20_knn.md
@@ -119,4 +119,4 @@ limit ${topn};
 | 8482  | 0.15229382 |
 
 
-Refer [this page](https://github.com/myui/hivemall/wiki/Efficient-Top-k-computation-on-Apache-Hive-using-Hivemall-UDTF#top-k-similarity-computation) for efficient top-k kNN computation.
\ No newline at end of file
+Refer [this page](../misc/topk.html#top-k-similarity-computation) for efficient top-k kNN computation.

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/a71bbb75/docs/gitbook/regression/e2006_arow.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/regression/e2006_arow.md b/docs/gitbook/regression/e2006_arow.md
index a02dfa8..abdb725 100644
--- a/docs/gitbook/regression/e2006_arow.md
+++ b/docs/gitbook/regression/e2006_arow.md
@@ -275,4 +275,4 @@ select
 from 
    e2006tfidf_arowe_submit;
 ```
-> 0.37789148212861856     0.14280197226536404     0.2357339155291536      0.5060283955470721
\ No newline at end of file
+> 0.37789148212861856     0.14280197226536404     0.2357339155291536      0.5060283955470721

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/a71bbb75/docs/gitbook/regression/kddcup12tr2_adagrad.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/regression/kddcup12tr2_adagrad.md b/docs/gitbook/regression/kddcup12tr2_adagrad.md
index f6c7675..1b82bd9 100644
--- a/docs/gitbook/regression/kddcup12tr2_adagrad.md
+++ b/docs/gitbook/regression/kddcup12tr2_adagrad.md
@@ -1,128 +1,128 @@
-<!--
-  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.
+<!--
+  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.
 -->
-        
-_Note adagrad/adadelta is supported from hivemall v0.3b2 or later (or in the master branch)._
-
-# Preparation 
-```sql
-add jar ./tmp/hivemall-with-dependencies.jar;
-source ./tmp/define-all.hive;
-
-use kdd12track2;
-
--- SET mapreduce.framework.name=yarn;
--- SET hive.execution.engine=mr;
--- SET mapreduce.framework.name=yarn-tez;
--- SET hive.execution.engine=tez;
-SET mapred.reduce.tasks=32; -- [optional] set the explicit number of reducers to make group-by aggregation faster
-```
-
-# AdaGrad
-```sql
-drop table adagrad_model;
-create table adagrad_model 
-as
-select 
- feature,
- avg(weight) as weight
-from 
- (select 
-     adagrad(features,label) as (feature,weight)
-  from 
-     training_orcfile
- ) t 
-group by feature;
-
-drop table adagrad_predict;
-create table adagrad_predict
-  ROW FORMAT DELIMITED 
-    FIELDS TERMINATED BY "\t"
-    LINES TERMINATED BY "\n"
-  STORED AS TEXTFILE
-as
-select
-  t.rowid, 
-  sigmoid(sum(m.weight)) as prob
-from 
-  testing_exploded  t LEFT OUTER JOIN
-  adagrad_model m ON (t.feature = m.feature)
-group by 
-  t.rowid
-order by 
-  rowid ASC;
-```
-
-```sh
-hadoop fs -getmerge /user/hive/warehouse/kdd12track2.db/adagrad_predict adagrad_predict.tbl
-
-gawk -F "\t" '{print $2;}' adagrad_predict.tbl > adagrad_predict.submit
-
-pypy scoreKDD.py KDD_Track2_solution.csv adagrad_predict.submit
-```
->AUC(SGD) : 0.739351
-
->AUC(ADAGRAD) : 0.743279
-
-# AdaDelta
-```sql
-drop table adadelta_model;
-create table adadelta_model 
-as
-select 
- feature,
- cast(avg(weight) as float) as weight
-from 
- (select 
-     adadelta(features,label) as (feature,weight)
-  from 
-     training_orcfile
- ) t 
-group by feature;
-
-drop table adadelta_predict;
-create table adadelta_predict
-  ROW FORMAT DELIMITED 
-    FIELDS TERMINATED BY "\t"
-    LINES TERMINATED BY "\n"
-  STORED AS TEXTFILE
-as
-select
-  t.rowid, 
-  sigmoid(sum(m.weight)) as prob
-from 
-  testing_exploded  t LEFT OUTER JOIN
-  adadelta_model m ON (t.feature = m.feature)
-group by 
-  t.rowid
-order by 
-  rowid ASC;
-```
-
-```sh
-hadoop fs -getmerge /user/hive/warehouse/kdd12track2.db/adadelta_predict adadelta_predict.tbl
-
-gawk -F "\t" '{print $2;}' adadelta_predict.tbl > adadelta_predict.submit
-
-pypy scoreKDD.py KDD_Track2_solution.csv adadelta_predict.submit
-```
->AUC(SGD) : 0.739351
-
->AUC(ADAGRAD) : 0.743279
-
-> AUC(AdaDelta) : 0.746878
\ No newline at end of file
+        
+_Note adagrad/adadelta is supported from hivemall v0.3b2 or later (or in the master branch)._
+
+# Preparation 
+```sql
+add jar ./tmp/hivemall-with-dependencies.jar;
+source ./tmp/define-all.hive;
+
+use kdd12track2;
+
+-- SET mapreduce.framework.name=yarn;
+-- SET hive.execution.engine=mr;
+-- SET mapreduce.framework.name=yarn-tez;
+-- SET hive.execution.engine=tez;
+SET mapred.reduce.tasks=32; -- [optional] set the explicit number of reducers to make group-by aggregation faster
+```
+
+# AdaGrad
+```sql
+drop table adagrad_model;
+create table adagrad_model 
+as
+select 
+ feature,
+ avg(weight) as weight
+from 
+ (select 
+     adagrad(features,label) as (feature,weight)
+  from 
+     training_orcfile
+ ) t 
+group by feature;
+
+drop table adagrad_predict;
+create table adagrad_predict
+  ROW FORMAT DELIMITED 
+    FIELDS TERMINATED BY "\t"
+    LINES TERMINATED BY "\n"
+  STORED AS TEXTFILE
+as
+select
+  t.rowid, 
+  sigmoid(sum(m.weight)) as prob
+from 
+  testing_exploded  t LEFT OUTER JOIN
+  adagrad_model m ON (t.feature = m.feature)
+group by 
+  t.rowid
+order by 
+  rowid ASC;
+```
+
+```sh
+hadoop fs -getmerge /user/hive/warehouse/kdd12track2.db/adagrad_predict adagrad_predict.tbl
+
+gawk -F "\t" '{print $2;}' adagrad_predict.tbl > adagrad_predict.submit
+
+pypy scoreKDD.py KDD_Track2_solution.csv adagrad_predict.submit
+```
+>AUC(SGD) : 0.739351
+
+>AUC(ADAGRAD) : 0.743279
+
+# AdaDelta
+```sql
+drop table adadelta_model;
+create table adadelta_model 
+as
+select 
+ feature,
+ cast(avg(weight) as float) as weight
+from 
+ (select 
+     adadelta(features,label) as (feature,weight)
+  from 
+     training_orcfile
+ ) t 
+group by feature;
+
+drop table adadelta_predict;
+create table adadelta_predict
+  ROW FORMAT DELIMITED 
+    FIELDS TERMINATED BY "\t"
+    LINES TERMINATED BY "\n"
+  STORED AS TEXTFILE
+as
+select
+  t.rowid, 
+  sigmoid(sum(m.weight)) as prob
+from 
+  testing_exploded  t LEFT OUTER JOIN
+  adadelta_model m ON (t.feature = m.feature)
+group by 
+  t.rowid
+order by 
+  rowid ASC;
+```
+
+```sh
+hadoop fs -getmerge /user/hive/warehouse/kdd12track2.db/adadelta_predict adadelta_predict.tbl
+
+gawk -F "\t" '{print $2;}' adadelta_predict.tbl > adadelta_predict.submit
+
+pypy scoreKDD.py KDD_Track2_solution.csv adadelta_predict.submit
+```
+>AUC(SGD) : 0.739351
+
+>AUC(ADAGRAD) : 0.743279
+
+> AUC(AdaDelta) : 0.746878

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/a71bbb75/docs/gitbook/regression/kddcup12tr2_dataset.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/regression/kddcup12tr2_dataset.md b/docs/gitbook/regression/kddcup12tr2_dataset.md
index 15bfbfd..c32958f 100644
--- a/docs/gitbook/regression/kddcup12tr2_dataset.md
+++ b/docs/gitbook/regression/kddcup12tr2_dataset.md
@@ -35,7 +35,7 @@ http://www.kddcup2012.org/c/kddcup2012-track2
 | training.txt | 9.9GB | 149,639,105 |
 | serid_profile.txt | 283MB | 23,669,283 |
 
-![tables](https://raw.github.com/myui/hivemall/master/resources/examples/kddtrack2/tables.png)
+![tables](../resources/images/kddtrack2tables.png)
 
 _Tokens are actually not used in this example. Try using them on your own._
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/a71bbb75/docs/gitbook/regression/kddcup12tr2_lr_amplify.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/regression/kddcup12tr2_lr_amplify.md b/docs/gitbook/regression/kddcup12tr2_lr_amplify.md
index e402ce4..5ede953 100644
--- a/docs/gitbook/regression/kddcup12tr2_lr_amplify.md
+++ b/docs/gitbook/regression/kddcup12tr2_lr_amplify.md
@@ -21,7 +21,7 @@ This article explains *amplify* technique that is useful for improving predictio
 
 Iterations are mandatory in machine learning (e.g., in [stochastic gradient descent](http://en.wikipedia.org/wiki/Stochastic_gradient_descent)) to get good prediction models. However, MapReduce is known to be not suited for iterative algorithms because IN/OUT of each MapReduce job is through HDFS.
 
-In this example, we show how Hivemall deals with this problem. We use [KDD Cup 2012, Track 2 Task](https://github.com/myui/hivemall/wiki/KDDCup-2012-track-2-CTR-prediction-dataset) as an example.
+In this example, we show how Hivemall deals with this problem. We use [KDD Cup 2012, Track 2 Task](../regression/kddcup12tr2_dataset.html) as an example.
 
 **WARNING**: rand_amplify() is supported in v0.2-beta1 and later.
 
@@ -73,7 +73,7 @@ The above query is executed by 2 MapReduce jobs as shown below:
 
 <img src="../resources/images/amplify.png" alt="amplifier"/>
 
-Using *trainning_x3*  instead of the plain training table results in higher and better AUC (0.746214) in [this](https://github.com/myui/hivemall/wiki/KDDCup-2012-track-2-CTR-prediction-(regression\)) example.
+Using *trainning_x3*  instead of the plain training table results in higher and better AUC (0.746214) in [this example](../regression/kddcup12tr2_lr.html#evaluation).
 
 A problem in amplify() is that the shuffle (copy) and merge phase of the stage 1 could become a bottleneck.
 When the training table is so large that involves 100 Map tasks, the merge operator needs to merge at least 100 files by (external) merge sort! 
@@ -108,7 +108,7 @@ The map-local multiplication and shuffling has no bottleneck in the merge phase
 
 <img src="../resources/images/randamplify_elapsed.png" alt="rand_amplify elapsed"/>
 
-Using *rand_amplify* results in a better AUC (0.743392) in [this](https://github.com/myui/hivemall/wiki/KDDCup-2012-track-2-CTR-prediction-(regression\)) example.
+Using *rand_amplify* results in a better AUC (0.743392) in [this example](../regression/kddcup12tr2_lr.html#evaluation).
 
 ---
 # Conclusion

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/a71bbb75/docs/gitbook/resources/images/kddtrack2tables.png
----------------------------------------------------------------------
diff --git a/docs/gitbook/resources/images/kddtrack2tables.png b/docs/gitbook/resources/images/kddtrack2tables.png
new file mode 100644
index 0000000..90012db
Binary files /dev/null and b/docs/gitbook/resources/images/kddtrack2tables.png differ

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/a71bbb75/docs/gitbook/tips/addbias.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/tips/addbias.md b/docs/gitbook/tips/addbias.md
index dfa4bfc..021ca64 100644
--- a/docs/gitbook/tips/addbias.md
+++ b/docs/gitbook/tips/addbias.md
@@ -28,7 +28,7 @@ Then, the predicted model considers bias existing in the dataset and the predict
 
 **addBias()** of Hivemall, adds a bias to a feature vector. 
 To enable a bias clause, use addBias() for **both**_(important!)_ training and test data as follows.
-The bias _b_ is a feature of "0" ("-1" in before v0.3) by the default. See [AddBiasUDF](https://github.com/myui/hivemall/blob/master/src/main/hivemall/ftvec/AddBiasUDF.java) for the detail.
+The bias _b_ is a feature of "0" ("-1" in before v0.3) by the default. See [AddBiasUDF](../tips/addbias.html) for the detail.
 
 Note that Bias is expressed as a feature that found in all training/testing examples.
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/a71bbb75/docs/gitbook/tips/emr.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/tips/emr.md b/docs/gitbook/tips/emr.md
index 61cb25b..049e6da 100644
--- a/docs/gitbook/tips/emr.md
+++ b/docs/gitbook/tips/emr.md
@@ -16,6 +16,8 @@
   specific language governing permissions and limitations
   under the License.
 -->
+
+<!-- toc -->
         
 ## Prerequisite
 Learn how to use Hive with Elastic MapReduce (EMR).  

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/a71bbb75/docs/gitbook/tips/hadoop_tuning.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/tips/hadoop_tuning.md b/docs/gitbook/tips/hadoop_tuning.md
index 7125508..507e19d 100644
--- a/docs/gitbook/tips/hadoop_tuning.md
+++ b/docs/gitbook/tips/hadoop_tuning.md
@@ -16,6 +16,8 @@
   specific language governing permissions and limitations
   under the License.
 -->
+
+<!-- toc -->
         
 # Prerequisites 
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/a71bbb75/docs/gitbook/tips/mixserver.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/tips/mixserver.md b/docs/gitbook/tips/mixserver.md
index bd58279..f9878e6 100644
--- a/docs/gitbook/tips/mixserver.md
+++ b/docs/gitbook/tips/mixserver.md
@@ -1,87 +1,86 @@
-<!--
-  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.
+<!--
+  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.
 -->
-        
-In this page, we will explain how to use model mixing on Hivemall. The model mixing is useful for a better prediction performance and faster convergence in training classifiers. 
-
-<!--
-You can find a brief explanation of the internal design of MIX protocol in [this slide](http://www.slideshare.net/myui/hivemall-mix-internal).
--->
-
-Prerequisite
-============
-
-* Hivemall v0.3 or later
-
-We recommend to use Mixing in a cluster with fast networking. The current standard GbE is enough though.
-
-Running Mix Server
-===================
-
-First, put the following files on server(s) that are accessible from Hadoop worker nodes:
-* [target/hivemall-mixserv.jar](https://github.com/myui/hivemall/releases)
-* [bin/run_mixserv.sh](https://github.com/myui/hivemall/raw/master/bin/run_mixserv.sh)
-
-_Caution: hivemall-mixserv.jar is large in size and thus only used for Mix servers._
-
-```sh
-# run a Mix Server
-./run_mixserv.sh
-```
-
-We assume in this example that Mix servers are running on host01, host03 and host03.
-The default port used by Mix server is 11212 and the port is configurable through "-port" option of run_mixserv.sh. 
-
-See [MixServer.java](https://github.com/myui/hivemall/blob/master/mixserv/src/main/java/hivemall/mix/server/MixServer.java#L90) to get detail of the Mix server options.
-
-We recommended to use multiple MIX servers to get better MIX throughput (3-5 or so would be enough for normal cluster size). The MIX protocol of Hivemall is *horizontally scalable* by adding MIX server nodes.
-
-Using Mix Protocol through Hivemall
-===================================
-
-[Install Hivemall](https://github.com/myui/hivemall/wiki/Installation) on Hive.
-
-_Make sure that [hivemall-with-dependencies.jar](https://github.com/myui/hivemall/raw/master/target/hivemall-with-dependencies.jar) is used for installation. The jar contains minimum requirement jars (netty,jsr305) for running Hivemall on Hive._
-
-Now, we explain that how to use mixing in [an example using KDD2010a dataset](https://github.com/myui/hivemall/wiki/KDD2010a-classification).
-
-Enabling the mixing on Hivemall is simple as follows:
-```sql
-use kdd2010;
-
-create table kdd10a_pa1_model1 as
-select 
- feature,
- cast(voted_avg(weight) as float) as weight
-from 
- (select 
-     train_pa1(addBias(features),label,"-mix host01,host02,host03") as (feature,weight)
-  from 
-     kdd10a_train_x3
- ) t 
-group by feature;
-```
-
-All you have to do is just adding "*-mix*" training option as seen in the above query.
-
-The effect of model mixing
-===========================
-
-In my experience, the MIX improved the prediction accuracy of the above KDD2010a PA1 training on a 32 nodes cluster from 0.844835019263103 (w/o mix) to 0.8678096499719774 (w/ mix).
-
+        
+In this page, we will explain how to use model mixing on Hivemall. The model mixing is useful for a better prediction performance and faster convergence in training classifiers. 
+You can find a brief explanation of the internal design of MIX protocol in [this slide](http://www.slideshare.net/myui/hivemall-mix-internal).
+
+<!-- toc -->
+
+Prerequisite
+============
+
+* Hivemall v0.3 or later
+
+    We recommend to use Mixing in a cluster with fast networking. The current standard GbE is enough though.
+
+Running Mix Server
+===================
+
+First, put the following files on server(s) that are accessible from Hadoop worker nodes:
+* [target/hivemall-mixserv.jar](https://github.com/myui/hivemall/releases)
+* [bin/run_mixserv.sh](https://github.com/myui/hivemall/raw/master/bin/run_mixserv.sh)
+
+_Caution: hivemall-mixserv.jar is large in size and thus only used for Mix servers._
+
+```sh
+# run a Mix Server
+./run_mixserv.sh
+```
+
+We assume in this example that Mix servers are running on host01, host03 and host03.
+The default port used by Mix server is 11212 and the port is configurable through "-port" option of run_mixserv.sh. 
+
+See [MixServer.java](https://github.com/myui/hivemall/blob/master/mixserv/src/main/java/hivemall/mix/server/MixServer.java#L90) to get detail of the Mix server options.
+
+We recommended to use multiple MIX servers to get better MIX throughput (3-5 or so would be enough for normal cluster size). The MIX protocol of Hivemall is *horizontally scalable* by adding MIX server nodes.
+
+Using Mix Protocol through Hivemall
+===================================
+
+[Install Hivemall](../getting_started/installation.html) on Hive.
+
+_Make sure that [hivemall-with-dependencies.jar](https://github.com/myui/hivemall/raw/master/target/hivemall-with-dependencies.jar) is used for installation. The jar contains minimum requirement jars (netty,jsr305) for running Hivemall on Hive._
+
+Now, we explain that how to use mixing in [an example using KDD2010a dataset](../binaryclass/kdd2010a_dataset.html).
+
+Enabling the mixing on Hivemall is simple as follows:
+```sql
+use kdd2010;
+
+create table kdd10a_pa1_model1 as
+select 
+ feature,
+ cast(voted_avg(weight) as float) as weight
+from 
+ (select 
+     train_pa1(addBias(features),label,"-mix host01,host02,host03") as (feature,weight)
+  from 
+     kdd10a_train_x3
+ ) t 
+group by feature;
+```
+
+All you have to do is just adding "*-mix*" training option as seen in the above query.
+
+The effect of model mixing
+===========================
+
+In my experience, the MIX improved the prediction accuracy of the above KDD2010a PA1 training on a 32 nodes cluster from 0.844835019263103 (w/o mix) to 0.8678096499719774 (w/ mix).
+
 The overhead of using the MIX protocol is *almost negligible* because the MIX communication is efficiently handled using asynchronous non-blocking I/O. Furthermore, the training time could be improved on certain settings because of the faster convergence due to mixing. 
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/a71bbb75/docs/gitbook/tips/rand_amplify.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/tips/rand_amplify.md b/docs/gitbook/tips/rand_amplify.md
index cd546ec..6d68dea 100644
--- a/docs/gitbook/tips/rand_amplify.md
+++ b/docs/gitbook/tips/rand_amplify.md
@@ -21,16 +21,16 @@ This article explains *amplify* technique that is useful for improving predictio
 
 Iterations are mandatory in machine learning (e.g., in [stochastic gradient descent](http://en.wikipedia.org/wiki/Stochastic_gradient_descent)) to get good prediction models. However, MapReduce is known to be not suited for iterative algorithms because IN/OUT of each MapReduce job is through HDFS.
 
-In this example, we show how Hivemall deals with this problem. We use [KDD Cup 2012, Track 2 Task](https://github.com/myui/hivemall/wiki/KDDCup-2012-track-2-CTR-prediction-dataset) as an example.
+In this example, we show how Hivemall deals with this problem. We use [KDD Cup 2012, Track 2 Task](../regression/kddcup12tr2_dataset.html) as an example.
 
-**WARNING**: rand_amplify() is supported in v0.2-beta1 and later.
+<!-- toc -->
 
 ---
 # Amplify training examples in Map phase and shuffle them in Reduce phase
 Hivemall provides the **amplify** UDTF to enumerate iteration effects in machine learning without several MapReduce steps. 
 
 The amplify function returns multiple rows for each row.
-The first argument ${xtimes} is the multiplication factor.  
+The first argument `${xtimes}` is the multiplication factor.  
 In the following examples, the multiplication factor is set to 3.
 
 ```sql
@@ -72,9 +72,9 @@ group by feature;
 The above query is executed by 2 MapReduce jobs as shown below:
 <img src="../resources/images/amplify.png" alt="amplifier"/>
 
-Using *trainning_x3*  instead of the plain training table results in higher and better AUC (0.746214) in [this](https://github.com/myui/hivemall/wiki/KDDCup-2012-track-2-CTR-prediction-(regression\)) example.
+Using *trainning_x3*  instead of the plain training table results in higher and better AUC (0.746214) in [this example](../regression/kddcup12tr2_lr_amplify.html#conclusion).
 
-A problem in amplify() is that the shuffle (copy) and merge phase of the stage 1 could become a bottleneck.
+A problem in `amplify()` is that the shuffle (copy) and merge phase of the stage 1 could become a bottleneck.
 When the training table is so large that involves 100 Map tasks, the merge operator needs to merge at least 100 files by (external) merge sort! 
 
 Note that the actual bottleneck is not M/R iterations but shuffling training instance. Iteration without shuffling (as in [the Spark example](http://spark.incubator.apache.org/examples.html)) causes very slow convergence and results in requiring more iterations. Shuffling cannot be avoided even in iterative MapReduce variants.
@@ -107,7 +107,7 @@ The map-local multiplication and shuffling has no bottleneck in the merge phase
 
 <img src="../resources/images/randamplify_elapsed.png" alt="randamplify_elapsed"/>
 
-Using *rand_amplify* results in a better AUC (0.743392) in [this](https://github.com/myui/hivemall/wiki/KDDCup-2012-track-2-CTR-prediction-(regression\)) example.
+Using *rand_amplify* results in a better AUC (0.743392) in [this example](../regression/kddcup12tr2_lr_amplify.html#conclusion).
 
 ---
 # Conclusion

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/a71bbb75/docs/gitbook/tips/rowid.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/tips/rowid.md b/docs/gitbook/tips/rowid.md
index 2b24401..ed6431e 100644
--- a/docs/gitbook/tips/rowid.md
+++ b/docs/gitbook/tips/rowid.md
@@ -16,7 +16,21 @@
   specific language governing permissions and limitations
   under the License.
 -->
-        
+
+<!-- toc -->
+
+# Rowid generator provided in Hivemall
+You can use [rowid() function](https://github.com/myui/hivemall/blob/master/src/main/java/hivemall/tools/mapred/RowIdUDF.java) to generate an unique rowid in Hivemall v0.2 or later.
+```sql
+select
+  rowid() as rowid, -- returns ${task_id}-${sequence_number}
+  *
+from 
+  xxx
+```
+
+# Other Rowid generation schemes using SQL
+
 ```sql
 CREATE TABLE xxx
 AS
@@ -37,14 +51,3 @@ select
   * 
 from a9atest;
 ```
-
-***
-# Rowid generator provided in Hivemall v0.2 or later
-You can use [rowid() function](https://github.com/myui/hivemall/blob/master/src/main/java/hivemall/tools/mapred/RowIdUDF.java) to generate an unique rowid in Hivemall v0.2 or later.
-```sql
-select
-  rowid() as rowid, -- returns ${task_id}-${sequence_number}
-  *
-from 
-  xxx
-```
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/a71bbb75/docs/gitbook/tips/rt_prediction.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/tips/rt_prediction.md b/docs/gitbook/tips/rt_prediction.md
index c342378..25d9ff7 100644
--- a/docs/gitbook/tips/rt_prediction.md
+++ b/docs/gitbook/tips/rt_prediction.md
@@ -16,23 +16,25 @@
   specific language governing permissions and limitations
   under the License.
 -->
-        
-Hivemall provides a batch learning scheme that builds prediction models on Apache Hive.
+
+Apache Hivemall provides a batch learning scheme that builds prediction models on Apache Hive.
 The learning process itself is a batch process; however, an online/real-time prediction can be achieved by carrying a prediction on a transactional relational DBMS.
 
 In this article, we explain how to run a real-time prediction using a relational DBMS. 
-We assume that you have already run the [a9a binary classification task](https://github.com/myui/hivemall/wiki#a9a-binary-classification).
+We assume that you have already run the [a9a binary classification task](../binaryclass/a9a.html).
+
+<!-- toc -->
 
 # Prerequisites
 
 - MySQL
 
-Put mysql-connector-java.jar (JDBC driver) on $SQOOP_HOME/lib.
+    Put mysql-connector-java.jar (JDBC driver) on $SQOOP_HOME/lib.
 
 - [Sqoop](http://sqoop.apache.org/)
 
-Sqoop 1.4.5 does not support Hadoop v2.6.0. So, you need to build packages for Hadoop 2.6.
-To do that you need to edit build.xml and ivy.xml as shown in [this patch](https://gist.github.com/myui/e8db4a31b574103133c6).
+    Sqoop 1.4.5 does not support Hadoop v2.6.0. So, you need to build packages for Hadoop 2.6.
+    To do that you need to edit build.xml and ivy.xml as shown in [this patch](https://gist.github.com/myui/e8db4a31b574103133c6).
 
 # Preparing Model Tables on MySQL
 
@@ -228,7 +230,7 @@ where
 1 row in set (0.00 sec)
 ```
 
-Similar to [the way in Hive](https://github.com/myui/hivemall/wiki/a9a-binary-classification-(logistic-regression)#prediction), you can run prediction as follows:
+Similar to [the way in Hive](../binaryclass/a9a_lr.html#prediction), you can run prediction as follows:
 
 ```sql
 select


[50/50] [abbrv] incubator-hivemall git commit: Update NOTICE

Posted by my...@apache.org.
Update NOTICE

Project: http://git-wip-us.apache.org/repos/asf/incubator-hivemall/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-hivemall/commit/72d6a629
Tree: http://git-wip-us.apache.org/repos/asf/incubator-hivemall/tree/72d6a629
Diff: http://git-wip-us.apache.org/repos/asf/incubator-hivemall/diff/72d6a629

Branch: refs/heads/master
Commit: 72d6a629f972abc2f38c63d20fe5c978618f8bf8
Parents: 69496fa
Author: Makoto YUI <yu...@gmail.com>
Authored: Mon Nov 28 15:29:52 2016 +0900
Committer: GitHub <no...@github.com>
Committed: Mon Nov 28 15:29:52 2016 +0900

----------------------------------------------------------------------
 NOTICE | 23 +++++++++++++++++++++--
 1 file changed, 21 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/72d6a629/NOTICE
----------------------------------------------------------------------
diff --git a/NOTICE b/NOTICE
index b252238..013774b 100644
--- a/NOTICE
+++ b/NOTICE
@@ -1,5 +1,5 @@
 Apache Hivemall
-Copyright 2016 The Apache Software Foundation
+Copyright 2016 and onwards The Apache Software Foundation
 
 This product includes software developed at
 The Apache Software Foundation (http://www.apache.org/).
@@ -33,11 +33,30 @@ o hivemall/core/src/main/java/hivemall/utils/collections/OpenHashMap.java
     https://github.com/slipperyseal/atomicobjects/
     Licensed under the Apache License, Version 2.0
 
+------------------------------------------------------------------------------------------------------
+Copyright notifications which have been relocated from ASF projects
+
+o hivemall/core/src/main/java/hivemall/utils/math/MathUtils.java#erfInv()
+
+    Copyright (C) 2003-2016 The Apache Software Foundation. 
+
+    http://commons.apache.org/proper/commons-math/
+    Licensed under the Apache License, Version 2.0
+
+o hivemall/core/src/main/java/hivemall/utils/buffer/DynamicByteArray.java
+
+    Copyright 2013-2015 The Apache Software Foundation
+
+    https://orc.apache.org/
+    Licensed under the Apache License, Version 2.0
+
 o hivemall/spark/spark-1.6/extra-src/hive/src/main/scala/org/apache/spark/sql/hive/HiveShim.scala
   hivemall/spark/spark-2.0/extra-src/hive/src/main/scala/org/apache/spark/sql/hive/HiveShim.scala
   hivemall/spark/spark-2.0/src/test/scala/org/apache/spark/sql/QueryTest.scala
   hivemall/spark/spark-2.0/src/test/scala/org/apache/spark/sql/catalyst/plans/PlanTest.scala
   hivemall/spark/spark-2.0/src/test/scala/org/apache/spark/sql/hive/test/TestHiveSingleton.scala
 
-    https://github.com/apache/spark
+    Copyright (C) 2014-2016 The Apache Software Foundation.
+
+    http://spark.apache.org/
     Licensed under the Apache License, Version 2.0


[25/50] [abbrv] incubator-hivemall git commit: Updated license headers of *.md

Posted by my...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/fc97a52e/docs/gitbook/regression/e2006_arow.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/regression/e2006_arow.md b/docs/gitbook/regression/e2006_arow.md
index a3b60eb..a02dfa8 100644
--- a/docs/gitbook/regression/e2006_arow.md
+++ b/docs/gitbook/regression/e2006_arow.md
@@ -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.
+-->
+        
 http://www.csie.ntu.edu.tw/~cjlin/libsvmtools/datasets/regression.html#E2006-tfidf
 
 ---

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/fc97a52e/docs/gitbook/regression/e2006_dataset.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/regression/e2006_dataset.md b/docs/gitbook/regression/e2006_dataset.md
index 329fb39..001eda2 100644
--- a/docs/gitbook/regression/e2006_dataset.md
+++ b/docs/gitbook/regression/e2006_dataset.md
@@ -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.
+-->
+        
 http://www.csie.ntu.edu.tw/~cjlin/libsvmtools/datasets/regression.html#E2006-tfidf
 
 Prerequisite

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/fc97a52e/docs/gitbook/regression/kddcup12tr2.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/regression/kddcup12tr2.md b/docs/gitbook/regression/kddcup12tr2.md
index e69de29..2959148 100644
--- a/docs/gitbook/regression/kddcup12tr2.md
+++ b/docs/gitbook/regression/kddcup12tr2.md
@@ -0,0 +1,19 @@
+<!--
+  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.
+-->
+        

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/fc97a52e/docs/gitbook/regression/kddcup12tr2_adagrad.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/regression/kddcup12tr2_adagrad.md b/docs/gitbook/regression/kddcup12tr2_adagrad.md
index e6c8eb4..f6c7675 100644
--- a/docs/gitbook/regression/kddcup12tr2_adagrad.md
+++ b/docs/gitbook/regression/kddcup12tr2_adagrad.md
@@ -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.
+-->
+        
 _Note adagrad/adadelta is supported from hivemall v0.3b2 or later (or in the master branch)._
 
 # Preparation 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/fc97a52e/docs/gitbook/regression/kddcup12tr2_dataset.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/regression/kddcup12tr2_dataset.md b/docs/gitbook/regression/kddcup12tr2_dataset.md
index 8713e99..15bfbfd 100644
--- a/docs/gitbook/regression/kddcup12tr2_dataset.md
+++ b/docs/gitbook/regression/kddcup12tr2_dataset.md
@@ -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.
+-->
+        
 The task is predicting the click through rate (CTR) of advertisement, meaning that we are to predict the probability of each ad being clicked. 
 http://www.kddcup2012.org/c/kddcup2012-track2
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/fc97a52e/docs/gitbook/regression/kddcup12tr2_lr.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/regression/kddcup12tr2_lr.md b/docs/gitbook/regression/kddcup12tr2_lr.md
index 0ff9b97..6db07ab 100644
--- a/docs/gitbook/regression/kddcup12tr2_lr.md
+++ b/docs/gitbook/regression/kddcup12tr2_lr.md
@@ -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.
+-->
+        
 The task is predicting the click through rate (CTR) of advertisement, meaning that we are to predict the probability of each ad being clicked.   
 http://www.kddcup2012.org/c/kddcup2012-track2
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/fc97a52e/docs/gitbook/regression/kddcup12tr2_lr_amplify.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/regression/kddcup12tr2_lr_amplify.md b/docs/gitbook/regression/kddcup12tr2_lr_amplify.md
index 4df124e..55b8caf 100644
--- a/docs/gitbook/regression/kddcup12tr2_lr_amplify.md
+++ b/docs/gitbook/regression/kddcup12tr2_lr_amplify.md
@@ -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.
+-->
+        
 This article explains *amplify* technique that is useful for improving prediction score.
 
 Iterations are mandatory in machine learning (e.g., in [stochastic gradient descent](http://en.wikipedia.org/wiki/Stochastic_gradient_descent)) to get good prediction models. However, MapReduce is known to be not suited for iterative algorithms because IN/OUT of each MapReduce job is through HDFS.

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/fc97a52e/docs/gitbook/tips/README.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/tips/README.md b/docs/gitbook/tips/README.md
index e69de29..2959148 100644
--- a/docs/gitbook/tips/README.md
+++ b/docs/gitbook/tips/README.md
@@ -0,0 +1,19 @@
+<!--
+  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.
+-->
+        

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/fc97a52e/docs/gitbook/tips/addbias.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/tips/addbias.md b/docs/gitbook/tips/addbias.md
index 2b11d51..dfa4bfc 100644
--- a/docs/gitbook/tips/addbias.md
+++ b/docs/gitbook/tips/addbias.md
@@ -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.
+-->
+        
 A trainer learns the function f(x)=y, or weights _W_, of the following form to predict a label y where x is a feature vector.
 _y=f(x)=Wx_
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/fc97a52e/docs/gitbook/tips/emr.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/tips/emr.md b/docs/gitbook/tips/emr.md
index 4521635..030a594 100644
--- a/docs/gitbook/tips/emr.md
+++ b/docs/gitbook/tips/emr.md
@@ -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.
+-->
+        
 ## Prerequisite
 Learn how to use Hive with Elastic MapReduce (EMR).  
 http://docs.aws.amazon.com/ElasticMapReduce/latest/DeveloperGuide/emr-hive.html

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/fc97a52e/docs/gitbook/tips/ensemble_learning.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/tips/ensemble_learning.md b/docs/gitbook/tips/ensemble_learning.md
index 6daaf1a..9288f84 100644
--- a/docs/gitbook/tips/ensemble_learning.md
+++ b/docs/gitbook/tips/ensemble_learning.md
@@ -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.
+-->
+        
 This example explains how to run ensemble learning in Hivemall.   
 Two heads are better than one? Let's verify it by ensemble learning.
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/fc97a52e/docs/gitbook/tips/general_tips.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/tips/general_tips.md b/docs/gitbook/tips/general_tips.md
index e69de29..2959148 100644
--- a/docs/gitbook/tips/general_tips.md
+++ b/docs/gitbook/tips/general_tips.md
@@ -0,0 +1,19 @@
+<!--
+  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.
+-->
+        

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/fc97a52e/docs/gitbook/tips/hadoop_tuning.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/tips/hadoop_tuning.md b/docs/gitbook/tips/hadoop_tuning.md
index a6c1854..7125508 100644
--- a/docs/gitbook/tips/hadoop_tuning.md
+++ b/docs/gitbook/tips/hadoop_tuning.md
@@ -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.
+-->
+        
 # Prerequisites 
 
 Please refer the following guides for Hadoop tuning:

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/fc97a52e/docs/gitbook/tips/mixserver.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/tips/mixserver.md b/docs/gitbook/tips/mixserver.md
index 631557c..bd58279 100644
--- a/docs/gitbook/tips/mixserver.md
+++ b/docs/gitbook/tips/mixserver.md
@@ -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.
+-->
+        
 In this page, we will explain how to use model mixing on Hivemall. The model mixing is useful for a better prediction performance and faster convergence in training classifiers. 
 
 <!--

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/fc97a52e/docs/gitbook/tips/rand_amplify.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/tips/rand_amplify.md b/docs/gitbook/tips/rand_amplify.md
index 4df124e..55b8caf 100644
--- a/docs/gitbook/tips/rand_amplify.md
+++ b/docs/gitbook/tips/rand_amplify.md
@@ -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.
+-->
+        
 This article explains *amplify* technique that is useful for improving prediction score.
 
 Iterations are mandatory in machine learning (e.g., in [stochastic gradient descent](http://en.wikipedia.org/wiki/Stochastic_gradient_descent)) to get good prediction models. However, MapReduce is known to be not suited for iterative algorithms because IN/OUT of each MapReduce job is through HDFS.

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/fc97a52e/docs/gitbook/tips/rowid.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/tips/rowid.md b/docs/gitbook/tips/rowid.md
index c43aa74..2b24401 100644
--- a/docs/gitbook/tips/rowid.md
+++ b/docs/gitbook/tips/rowid.md
@@ -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.
+-->
+        
 ```sql
 CREATE TABLE xxx
 AS

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/fc97a52e/docs/gitbook/tips/rt_prediction.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/tips/rt_prediction.md b/docs/gitbook/tips/rt_prediction.md
index 3ac4fb6..c342378 100644
--- a/docs/gitbook/tips/rt_prediction.md
+++ b/docs/gitbook/tips/rt_prediction.md
@@ -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.
+-->
+        
 Hivemall provides a batch learning scheme that builds prediction models on Apache Hive.
 The learning process itself is a batch process; however, an online/real-time prediction can be achieved by carrying a prediction on a transactional relational DBMS.
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/fc97a52e/docs/gitbook/troubleshooting/README.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/troubleshooting/README.md b/docs/gitbook/troubleshooting/README.md
index e69de29..2959148 100644
--- a/docs/gitbook/troubleshooting/README.md
+++ b/docs/gitbook/troubleshooting/README.md
@@ -0,0 +1,19 @@
+<!--
+  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.
+-->
+        

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/fc97a52e/docs/gitbook/troubleshooting/asterisk.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/troubleshooting/asterisk.md b/docs/gitbook/troubleshooting/asterisk.md
index 49e2f71..621ab3f 100644
--- a/docs/gitbook/troubleshooting/asterisk.md
+++ b/docs/gitbook/troubleshooting/asterisk.md
@@ -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.
+-->
+        
 See [HIVE-4181](https://issues.apache.org/jira/browse/HIVE-4181) that asterisk argument without table alias for UDTF is not working. It has been fixed as part of Hive v0.12 release.
 
 A possible workaround is to use asterisk with a table alias, or to specify names of arguments explicitly.
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/fc97a52e/docs/gitbook/troubleshooting/mapjoin_classcastex.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/troubleshooting/mapjoin_classcastex.md b/docs/gitbook/troubleshooting/mapjoin_classcastex.md
index c48919a..28e7709 100644
--- a/docs/gitbook/troubleshooting/mapjoin_classcastex.md
+++ b/docs/gitbook/troubleshooting/mapjoin_classcastex.md
@@ -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.
+-->
+        
 Map-side join on Tez causes [ClassCastException](http://markmail.org/message/7cwbgupnhah6ggkv) when a serialized table contains array column(s).
 
 [Workaround] Try setting _hive.mapjoin.optimized.hashtable_ off as follows:

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/fc97a52e/docs/gitbook/troubleshooting/mapjoin_task_error.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/troubleshooting/mapjoin_task_error.md b/docs/gitbook/troubleshooting/mapjoin_task_error.md
index 02aff2f..78b4e32 100644
--- a/docs/gitbook/troubleshooting/mapjoin_task_error.md
+++ b/docs/gitbook/troubleshooting/mapjoin_task_error.md
@@ -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 Hive 0.11.0, **hive.auto.convert.join** is [enabled by the default](https://issues.apache.org/jira/browse/HIVE-3297).
 
 When using complex queries using views, the auto conversion sometimes throws SemanticException, cannot serialize object.

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/fc97a52e/docs/gitbook/troubleshooting/num_mappers.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/troubleshooting/num_mappers.md b/docs/gitbook/troubleshooting/num_mappers.md
index be01f2a..c1820db 100644
--- a/docs/gitbook/troubleshooting/num_mappers.md
+++ b/docs/gitbook/troubleshooting/num_mappers.md
@@ -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.
+-->
+        
 The default _hive.input.format_ is set to _org.apache.hadoop.hive.ql.io.CombineHiveInputFormat_.
 This configuration could give less number of mappers than the split size (i.e., # blocks in HDFS) of the input table.
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/fc97a52e/docs/gitbook/troubleshooting/oom.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/troubleshooting/oom.md b/docs/gitbook/troubleshooting/oom.md
index 643d09a..50bee25 100644
--- a/docs/gitbook/troubleshooting/oom.md
+++ b/docs/gitbook/troubleshooting/oom.md
@@ -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.
+-->
+        
 # OOM in mappers
 
 In a certain setting, the default input split size is too large for Hivemall. Due to that, OutOfMemoryError cloud happen on mappers in the middle of training.

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/fc97a52e/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index f36a36c..6616f47 100644
--- a/pom.xml
+++ b/pom.xml
@@ -391,17 +391,22 @@
 							<currentYear>${build.year}</currentYear>
 							<copyrightOwner>${project.organization.name}</copyrightOwner>
 						</properties>
+						<useDefaultExcludes>false</useDefaultExcludes>
 						<includes>
 							<include>src/main/**/*.java</include>
 							<include>src/test/**/*.java</include>
 							<include>src/main/**/*.scala</include>
 							<include>src/test/**/*.scala</include>
 							<include>bin/*.sh</include>
+							<include>docs/gitbook/**/*.md</include>
 						</includes>
 						<encoding>UTF-8</encoding>
 						<headerDefinitions>
 							<headerDefinition>${main.basedir}/resources/header-definition.xml</headerDefinition>
 						</headerDefinitions>
+						<mapping>
+						  <md>XML_STYLE</md>
+						</mapping>
 					</configuration>
 					<executions>
 						<execution>

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/fc97a52e/resources/header-definition.xml
----------------------------------------------------------------------
diff --git a/resources/header-definition.xml b/resources/header-definition.xml
index 057ad52..232b1d7 100644
--- a/resources/header-definition.xml
+++ b/resources/header-definition.xml
@@ -11,4 +11,16 @@
         <isMultiline>true</isMultiline>
         <padLines>false</padLines>
     </javadoc_style>
+    <xml_style>
+        <firstLine><![CDATA[<!--]]></firstLine>
+        <beforeEachLine>  </beforeEachLine>
+        <endLine><![CDATA[-->
+        ]]></endLine>
+        <skipLine><![CDATA[^<\?xml.*>$]]></skipLine>
+        <firstLineDetectionPattern><![CDATA[(\s|\t)*<!--.*$]]></firstLineDetectionPattern>
+        <lastLineDetectionPattern><![CDATA[.*-->(\s|\t)*$]]></lastLineDetectionPattern>
+        <allowBlankLines>false</allowBlankLines>
+        <isMultiline>true</isMultiline>
+        <padLines>true</padLines>
+    </xml_style>
 </additionalHeaders>


[12/50] [abbrv] incubator-hivemall git commit: Updated license headers

Posted by my...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/spark/spark-1.6/src/test/scala/org/apache/spark/test/TestUtils.scala
----------------------------------------------------------------------
diff --git a/spark/spark-1.6/src/test/scala/org/apache/spark/test/TestUtils.scala b/spark/spark-1.6/src/test/scala/org/apache/spark/test/TestUtils.scala
index 7f20d7e..9c77fae 100644
--- a/spark/spark-1.6/src/test/scala/org/apache/spark/test/TestUtils.scala
+++ b/spark/spark-1.6/src/test/scala/org/apache/spark/test/TestUtils.scala
@@ -1,20 +1,21 @@
 /*
- * 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
+ * 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
+ *   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.
+ * 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 org.apache.spark.test
 
 import scala.reflect.runtime.{universe => ru}

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/spark/spark-2.0/extra-src/hive/src/main/scala/org/apache/spark/sql/hive/HiveShim.scala
----------------------------------------------------------------------
diff --git a/spark/spark-2.0/extra-src/hive/src/main/scala/org/apache/spark/sql/hive/HiveShim.scala b/spark/spark-2.0/extra-src/hive/src/main/scala/org/apache/spark/sql/hive/HiveShim.scala
index 0d2a765..a695841 100644
--- a/spark/spark-2.0/extra-src/hive/src/main/scala/org/apache/spark/sql/hive/HiveShim.scala
+++ b/spark/spark-2.0/extra-src/hive/src/main/scala/org/apache/spark/sql/hive/HiveShim.scala
@@ -1,20 +1,21 @@
 /*
- * 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
+ * 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
+ *   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.
+ * 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 org.apache.spark.sql.hive
 
 import java.io.{InputStream, OutputStream}

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/spark/spark-2.0/pom.xml
----------------------------------------------------------------------
diff --git a/spark/spark-2.0/pom.xml b/spark/spark-2.0/pom.xml
index 031d602..79a43de 100644
--- a/spark/spark-2.0/pom.xml
+++ b/spark/spark-2.0/pom.xml
@@ -17,6 +17,7 @@
 		<PermGen>64m</PermGen>
 		<MaxPermGen>1024m</MaxPermGen>
 		<CodeCacheSize>512m</CodeCacheSize>
+		<main.basedir>${project.parent.basedir}</main.basedir>
 	</properties>
 
 	<dependencies>

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/spark/spark-2.0/src/main/java/hivemall/xgboost/XGBoostOptions.scala
----------------------------------------------------------------------
diff --git a/spark/spark-2.0/src/main/java/hivemall/xgboost/XGBoostOptions.scala b/spark/spark-2.0/src/main/java/hivemall/xgboost/XGBoostOptions.scala
index 6189dd1..48a773b 100644
--- a/spark/spark-2.0/src/main/java/hivemall/xgboost/XGBoostOptions.scala
+++ b/spark/spark-2.0/src/main/java/hivemall/xgboost/XGBoostOptions.scala
@@ -1,20 +1,21 @@
 /*
- * 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
+ * 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
+ *   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.
+ * 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.xgboost
 
 import scala.collection.mutable

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/spark/spark-2.0/src/main/java/hivemall/xgboost/classification/XGBoostBinaryClassifierUDTFWrapper.java
----------------------------------------------------------------------
diff --git a/spark/spark-2.0/src/main/java/hivemall/xgboost/classification/XGBoostBinaryClassifierUDTFWrapper.java b/spark/spark-2.0/src/main/java/hivemall/xgboost/classification/XGBoostBinaryClassifierUDTFWrapper.java
index c6c2127..310d15e 100644
--- a/spark/spark-2.0/src/main/java/hivemall/xgboost/classification/XGBoostBinaryClassifierUDTFWrapper.java
+++ b/spark/spark-2.0/src/main/java/hivemall/xgboost/classification/XGBoostBinaryClassifierUDTFWrapper.java
@@ -1,20 +1,21 @@
 /*
- * 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
+ * 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
+ *   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.
+ * 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.xgboost.classification;
 
 import java.util.UUID;

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/spark/spark-2.0/src/main/java/hivemall/xgboost/classification/XGBoostMulticlassClassifierUDTFWrapper.java
----------------------------------------------------------------------
diff --git a/spark/spark-2.0/src/main/java/hivemall/xgboost/classification/XGBoostMulticlassClassifierUDTFWrapper.java b/spark/spark-2.0/src/main/java/hivemall/xgboost/classification/XGBoostMulticlassClassifierUDTFWrapper.java
index fbc4652..81e6fe8 100644
--- a/spark/spark-2.0/src/main/java/hivemall/xgboost/classification/XGBoostMulticlassClassifierUDTFWrapper.java
+++ b/spark/spark-2.0/src/main/java/hivemall/xgboost/classification/XGBoostMulticlassClassifierUDTFWrapper.java
@@ -1,20 +1,21 @@
 /*
- * 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
+ * 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
+ *   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.
+ * 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.xgboost.classification;
 
 import java.util.UUID;

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/spark/spark-2.0/src/main/java/hivemall/xgboost/package.scala
----------------------------------------------------------------------
diff --git a/spark/spark-2.0/src/main/java/hivemall/xgboost/package.scala b/spark/spark-2.0/src/main/java/hivemall/xgboost/package.scala
index 720f760..2624412 100644
--- a/spark/spark-2.0/src/main/java/hivemall/xgboost/package.scala
+++ b/spark/spark-2.0/src/main/java/hivemall/xgboost/package.scala
@@ -1,22 +1,21 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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
 
 import org.apache.spark.sql.hive.source.XGBoostFileFormat

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/spark/spark-2.0/src/main/java/hivemall/xgboost/regression/XGBoostRegressionUDTFWrapper.java
----------------------------------------------------------------------
diff --git a/spark/spark-2.0/src/main/java/hivemall/xgboost/regression/XGBoostRegressionUDTFWrapper.java b/spark/spark-2.0/src/main/java/hivemall/xgboost/regression/XGBoostRegressionUDTFWrapper.java
index bfc1a12..b72e045 100644
--- a/spark/spark-2.0/src/main/java/hivemall/xgboost/regression/XGBoostRegressionUDTFWrapper.java
+++ b/spark/spark-2.0/src/main/java/hivemall/xgboost/regression/XGBoostRegressionUDTFWrapper.java
@@ -1,20 +1,21 @@
 /*
- * 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
+ * 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
+ *   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.
+ * 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.xgboost.regression;
 
 import java.util.UUID;

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/spark/spark-2.0/src/main/scala/hivemall/tools/RegressionDatagen.scala
----------------------------------------------------------------------
diff --git a/spark/spark-2.0/src/main/scala/hivemall/tools/RegressionDatagen.scala b/spark/spark-2.0/src/main/scala/hivemall/tools/RegressionDatagen.scala
index 9b90b98..01664f4 100644
--- a/spark/spark-2.0/src/main/scala/hivemall/tools/RegressionDatagen.scala
+++ b/spark/spark-2.0/src/main/scala/hivemall/tools/RegressionDatagen.scala
@@ -1,20 +1,21 @@
 /*
- * 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
+ * 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
+ *   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.
+ * 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.tools
 
 import org.apache.spark.sql.{DataFrame, Row, SQLContext}

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/spark/spark-2.0/src/main/scala/org/apache/spark/sql/catalyst/expressions/EachTopK.scala
----------------------------------------------------------------------
diff --git a/spark/spark-2.0/src/main/scala/org/apache/spark/sql/catalyst/expressions/EachTopK.scala b/spark/spark-2.0/src/main/scala/org/apache/spark/sql/catalyst/expressions/EachTopK.scala
index d0392c6..491363d 100644
--- a/spark/spark-2.0/src/main/scala/org/apache/spark/sql/catalyst/expressions/EachTopK.scala
+++ b/spark/spark-2.0/src/main/scala/org/apache/spark/sql/catalyst/expressions/EachTopK.scala
@@ -1,20 +1,21 @@
 /*
- * 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
+ * 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
+ *   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.
+ * 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 org.apache.spark.sql.catalyst.expressions
 
 import org.apache.spark.sql.catalyst.InternalRow

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/spark/spark-2.0/src/main/scala/org/apache/spark/sql/hive/HivemallGroupedDataset.scala
----------------------------------------------------------------------
diff --git a/spark/spark-2.0/src/main/scala/org/apache/spark/sql/hive/HivemallGroupedDataset.scala b/spark/spark-2.0/src/main/scala/org/apache/spark/sql/hive/HivemallGroupedDataset.scala
index 654319e..8ac7185 100644
--- a/spark/spark-2.0/src/main/scala/org/apache/spark/sql/hive/HivemallGroupedDataset.scala
+++ b/spark/spark-2.0/src/main/scala/org/apache/spark/sql/hive/HivemallGroupedDataset.scala
@@ -1,20 +1,21 @@
 /*
- * 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
+ * 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
+ *   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.
+ * 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 org.apache.spark.sql.hive
 
 import org.apache.spark.sql.AnalysisException

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/spark/spark-2.0/src/main/scala/org/apache/spark/sql/hive/HivemallOps.scala
----------------------------------------------------------------------
diff --git a/spark/spark-2.0/src/main/scala/org/apache/spark/sql/hive/HivemallOps.scala b/spark/spark-2.0/src/main/scala/org/apache/spark/sql/hive/HivemallOps.scala
index 2a78ea8..ba58039 100644
--- a/spark/spark-2.0/src/main/scala/org/apache/spark/sql/hive/HivemallOps.scala
+++ b/spark/spark-2.0/src/main/scala/org/apache/spark/sql/hive/HivemallOps.scala
@@ -1,20 +1,21 @@
 /*
- * 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
+ * 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
+ *   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.
+ * 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 org.apache.spark.sql.hive
 
 import java.util.UUID

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/spark/spark-2.0/src/main/scala/org/apache/spark/sql/hive/HivemallUtils.scala
----------------------------------------------------------------------
diff --git a/spark/spark-2.0/src/main/scala/org/apache/spark/sql/hive/HivemallUtils.scala b/spark/spark-2.0/src/main/scala/org/apache/spark/sql/hive/HivemallUtils.scala
index 290cadb..6924347 100644
--- a/spark/spark-2.0/src/main/scala/org/apache/spark/sql/hive/HivemallUtils.scala
+++ b/spark/spark-2.0/src/main/scala/org/apache/spark/sql/hive/HivemallUtils.scala
@@ -1,20 +1,21 @@
 /*
- * 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
+ * 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
+ *   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.
+ * 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 org.apache.spark.sql.hive
 
 import org.apache.spark.ml.linalg.{DenseVector => SDV, SparseVector => SSV, Vector => SV}

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/spark/spark-2.0/src/main/scala/org/apache/spark/sql/hive/source/XGBoostFileFormat.scala
----------------------------------------------------------------------
diff --git a/spark/spark-2.0/src/main/scala/org/apache/spark/sql/hive/source/XGBoostFileFormat.scala b/spark/spark-2.0/src/main/scala/org/apache/spark/sql/hive/source/XGBoostFileFormat.scala
index 8678b0d..9cdc09f 100644
--- a/spark/spark-2.0/src/main/scala/org/apache/spark/sql/hive/source/XGBoostFileFormat.scala
+++ b/spark/spark-2.0/src/main/scala/org/apache/spark/sql/hive/source/XGBoostFileFormat.scala
@@ -1,20 +1,21 @@
 /*
- * 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
+ * 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
+ *   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.
+ * 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 org.apache.spark.sql.hive.source
 
 import java.io.File

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/spark/spark-2.0/src/test/scala/hivemall/mix/server/MixServerSuite.scala
----------------------------------------------------------------------
diff --git a/spark/spark-2.0/src/test/scala/hivemall/mix/server/MixServerSuite.scala b/spark/spark-2.0/src/test/scala/hivemall/mix/server/MixServerSuite.scala
index cb3abee..dbb818b 100644
--- a/spark/spark-2.0/src/test/scala/hivemall/mix/server/MixServerSuite.scala
+++ b/spark/spark-2.0/src/test/scala/hivemall/mix/server/MixServerSuite.scala
@@ -1,20 +1,21 @@
 /*
- * 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
+ * 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
+ *   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.
+ * 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.mix.server
 
 import java.util.Random

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/spark/spark-2.0/src/test/scala/hivemall/tools/RegressionDatagenSuite.scala
----------------------------------------------------------------------
diff --git a/spark/spark-2.0/src/test/scala/hivemall/tools/RegressionDatagenSuite.scala b/spark/spark-2.0/src/test/scala/hivemall/tools/RegressionDatagenSuite.scala
index 379b3da..8c06837 100644
--- a/spark/spark-2.0/src/test/scala/hivemall/tools/RegressionDatagenSuite.scala
+++ b/spark/spark-2.0/src/test/scala/hivemall/tools/RegressionDatagenSuite.scala
@@ -1,20 +1,21 @@
 /*
- * 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
+ * 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
+ *   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.
+ * 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.tools
 
 import org.scalatest.FunSuite

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/spark/spark-2.0/src/test/scala/org/apache/spark/SparkFunSuite.scala
----------------------------------------------------------------------
diff --git a/spark/spark-2.0/src/test/scala/org/apache/spark/SparkFunSuite.scala b/spark/spark-2.0/src/test/scala/org/apache/spark/SparkFunSuite.scala
index 55794f5..0b101c8 100644
--- a/spark/spark-2.0/src/test/scala/org/apache/spark/SparkFunSuite.scala
+++ b/spark/spark-2.0/src/test/scala/org/apache/spark/SparkFunSuite.scala
@@ -1,20 +1,21 @@
 /*
- * 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
+ * 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
+ *   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.
+ * 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 org.apache.spark
 
 // scalastyle:off

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/spark/spark-2.0/src/test/scala/org/apache/spark/ml/feature/HivemallLabeledPointSuite.scala
----------------------------------------------------------------------
diff --git a/spark/spark-2.0/src/test/scala/org/apache/spark/ml/feature/HivemallLabeledPointSuite.scala b/spark/spark-2.0/src/test/scala/org/apache/spark/ml/feature/HivemallLabeledPointSuite.scala
index f9d207e..f57983f 100644
--- a/spark/spark-2.0/src/test/scala/org/apache/spark/ml/feature/HivemallLabeledPointSuite.scala
+++ b/spark/spark-2.0/src/test/scala/org/apache/spark/ml/feature/HivemallLabeledPointSuite.scala
@@ -1,20 +1,21 @@
 /*
- * 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
+ * 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
+ *   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.
+ * 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 org.apache.spark.ml.feature
 
 import org.apache.spark.SparkFunSuite

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/spark/spark-2.0/src/test/scala/org/apache/spark/sql/QueryTest.scala
----------------------------------------------------------------------
diff --git a/spark/spark-2.0/src/test/scala/org/apache/spark/sql/QueryTest.scala b/spark/spark-2.0/src/test/scala/org/apache/spark/sql/QueryTest.scala
index cad0a71..8b03911 100644
--- a/spark/spark-2.0/src/test/scala/org/apache/spark/sql/QueryTest.scala
+++ b/spark/spark-2.0/src/test/scala/org/apache/spark/sql/QueryTest.scala
@@ -1,20 +1,21 @@
 /*
- * 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
+ * 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
+ *   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.
+ * 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 org.apache.spark.sql
 
 import java.util.{ArrayDeque, Locale, TimeZone}

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/spark/spark-2.0/src/test/scala/org/apache/spark/sql/catalyst/plans/PlanTest.scala
----------------------------------------------------------------------
diff --git a/spark/spark-2.0/src/test/scala/org/apache/spark/sql/catalyst/plans/PlanTest.scala b/spark/spark-2.0/src/test/scala/org/apache/spark/sql/catalyst/plans/PlanTest.scala
index 754d0bc..816576e 100644
--- a/spark/spark-2.0/src/test/scala/org/apache/spark/sql/catalyst/plans/PlanTest.scala
+++ b/spark/spark-2.0/src/test/scala/org/apache/spark/sql/catalyst/plans/PlanTest.scala
@@ -1,20 +1,21 @@
 /*
- * 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
+ * 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
+ *   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.
+ * 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 org.apache.spark.sql.catalyst.plans
 
 import org.apache.spark.SparkFunSuite

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/spark/spark-2.0/src/test/scala/org/apache/spark/sql/hive/HiveUdfSuite.scala
----------------------------------------------------------------------
diff --git a/spark/spark-2.0/src/test/scala/org/apache/spark/sql/hive/HiveUdfSuite.scala b/spark/spark-2.0/src/test/scala/org/apache/spark/sql/hive/HiveUdfSuite.scala
index 576d47b..f8622c6 100644
--- a/spark/spark-2.0/src/test/scala/org/apache/spark/sql/hive/HiveUdfSuite.scala
+++ b/spark/spark-2.0/src/test/scala/org/apache/spark/sql/hive/HiveUdfSuite.scala
@@ -1,20 +1,21 @@
 /*
- * 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
+ * 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
+ *   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.
+ * 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 org.apache.spark.sql.hive
 
 import org.apache.spark.sql.Row

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/spark/spark-2.0/src/test/scala/org/apache/spark/sql/hive/HivemallOpsSuite.scala
----------------------------------------------------------------------
diff --git a/spark/spark-2.0/src/test/scala/org/apache/spark/sql/hive/HivemallOpsSuite.scala b/spark/spark-2.0/src/test/scala/org/apache/spark/sql/hive/HivemallOpsSuite.scala
index 8a7b467..a093e07 100644
--- a/spark/spark-2.0/src/test/scala/org/apache/spark/sql/hive/HivemallOpsSuite.scala
+++ b/spark/spark-2.0/src/test/scala/org/apache/spark/sql/hive/HivemallOpsSuite.scala
@@ -1,20 +1,21 @@
 /*
- * 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
+ * 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
+ *   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.
+ * 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 org.apache.spark.sql.hive
 
 import org.apache.spark.sql.{AnalysisException, Column, Row}

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/spark/spark-2.0/src/test/scala/org/apache/spark/sql/hive/ModelMixingSuite.scala
----------------------------------------------------------------------
diff --git a/spark/spark-2.0/src/test/scala/org/apache/spark/sql/hive/ModelMixingSuite.scala b/spark/spark-2.0/src/test/scala/org/apache/spark/sql/hive/ModelMixingSuite.scala
index f38cd9c..5a58c2d 100644
--- a/spark/spark-2.0/src/test/scala/org/apache/spark/sql/hive/ModelMixingSuite.scala
+++ b/spark/spark-2.0/src/test/scala/org/apache/spark/sql/hive/ModelMixingSuite.scala
@@ -1,20 +1,21 @@
 /*
- * 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
+ * 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
+ *   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.
+ * 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 org.apache.spark.sql.hive
 
 import java.io.{BufferedInputStream, BufferedReader, InputStream, InputStreamReader}

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/spark/spark-2.0/src/test/scala/org/apache/spark/sql/hive/XGBoostSuite.scala
----------------------------------------------------------------------
diff --git a/spark/spark-2.0/src/test/scala/org/apache/spark/sql/hive/XGBoostSuite.scala b/spark/spark-2.0/src/test/scala/org/apache/spark/sql/hive/XGBoostSuite.scala
index 486f275..136d6bc 100644
--- a/spark/spark-2.0/src/test/scala/org/apache/spark/sql/hive/XGBoostSuite.scala
+++ b/spark/spark-2.0/src/test/scala/org/apache/spark/sql/hive/XGBoostSuite.scala
@@ -1,20 +1,21 @@
 /*
- * 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
+ * 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
+ *   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.
+ * 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 org.apache.spark.sql.hive
 
 import java.io.File

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/spark/spark-2.0/src/test/scala/org/apache/spark/sql/hive/benchmark/MiscBenchmark.scala
----------------------------------------------------------------------
diff --git a/spark/spark-2.0/src/test/scala/org/apache/spark/sql/hive/benchmark/MiscBenchmark.scala b/spark/spark-2.0/src/test/scala/org/apache/spark/sql/hive/benchmark/MiscBenchmark.scala
index 97f296e..a16ebb8 100644
--- a/spark/spark-2.0/src/test/scala/org/apache/spark/sql/hive/benchmark/MiscBenchmark.scala
+++ b/spark/spark-2.0/src/test/scala/org/apache/spark/sql/hive/benchmark/MiscBenchmark.scala
@@ -1,20 +1,21 @@
 /*
- * 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
+ * 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
+ *   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.
+ * 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 org.apache.spark.sql.hive.benchmark
 
 import org.apache.spark.SparkFunSuite

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/spark/spark-2.0/src/test/scala/org/apache/spark/sql/hive/test/TestHiveSingleton.scala
----------------------------------------------------------------------
diff --git a/spark/spark-2.0/src/test/scala/org/apache/spark/sql/hive/test/TestHiveSingleton.scala b/spark/spark-2.0/src/test/scala/org/apache/spark/sql/hive/test/TestHiveSingleton.scala
index a19c6bd..eeb32dc 100644
--- a/spark/spark-2.0/src/test/scala/org/apache/spark/sql/hive/test/TestHiveSingleton.scala
+++ b/spark/spark-2.0/src/test/scala/org/apache/spark/sql/hive/test/TestHiveSingleton.scala
@@ -1,20 +1,21 @@
 /*
- * 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
+ * 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
+ *   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.
+ * 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 org.apache.spark.sql.hive.test
 
 import org.scalatest.BeforeAndAfterAll

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/spark/spark-2.0/src/test/scala/org/apache/spark/streaming/HivemallFeatureOpsSuite.scala
----------------------------------------------------------------------
diff --git a/spark/spark-2.0/src/test/scala/org/apache/spark/streaming/HivemallFeatureOpsSuite.scala b/spark/spark-2.0/src/test/scala/org/apache/spark/streaming/HivemallFeatureOpsSuite.scala
index 64dec6b..0285c4e 100644
--- a/spark/spark-2.0/src/test/scala/org/apache/spark/streaming/HivemallFeatureOpsSuite.scala
+++ b/spark/spark-2.0/src/test/scala/org/apache/spark/streaming/HivemallFeatureOpsSuite.scala
@@ -1,20 +1,21 @@
 /*
- * 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
+ * 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
+ *   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.
+ * 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 org.apache.spark.streaming
 
 import reflect.ClassTag

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/spark/spark-2.0/src/test/scala/org/apache/spark/test/HivemallFeatureQueryTest.scala
----------------------------------------------------------------------
diff --git a/spark/spark-2.0/src/test/scala/org/apache/spark/test/HivemallFeatureQueryTest.scala b/spark/spark-2.0/src/test/scala/org/apache/spark/test/HivemallFeatureQueryTest.scala
index 40342bc..417d080 100644
--- a/spark/spark-2.0/src/test/scala/org/apache/spark/test/HivemallFeatureQueryTest.scala
+++ b/spark/spark-2.0/src/test/scala/org/apache/spark/test/HivemallFeatureQueryTest.scala
@@ -1,20 +1,21 @@
 /*
- * 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
+ * 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
+ *   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.
+ * 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 org.apache.spark.test
 
 import scala.collection.mutable.Seq

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/spark/spark-2.0/src/test/scala/org/apache/spark/test/TestUtils.scala
----------------------------------------------------------------------
diff --git a/spark/spark-2.0/src/test/scala/org/apache/spark/test/TestUtils.scala b/spark/spark-2.0/src/test/scala/org/apache/spark/test/TestUtils.scala
index 2e3bb27..6650a93 100644
--- a/spark/spark-2.0/src/test/scala/org/apache/spark/test/TestUtils.scala
+++ b/spark/spark-2.0/src/test/scala/org/apache/spark/test/TestUtils.scala
@@ -1,20 +1,21 @@
 /*
- * 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
+ * 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
+ *   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.
+ * 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 org.apache.spark.test
 
 import scala.reflect.runtime.{universe => ru}

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/spark/spark-2.0/src/test/scala/org/apache/spark/test/VectorQueryTest.scala
----------------------------------------------------------------------
diff --git a/spark/spark-2.0/src/test/scala/org/apache/spark/test/VectorQueryTest.scala b/spark/spark-2.0/src/test/scala/org/apache/spark/test/VectorQueryTest.scala
index 411689e..e1384d1 100644
--- a/spark/spark-2.0/src/test/scala/org/apache/spark/test/VectorQueryTest.scala
+++ b/spark/spark-2.0/src/test/scala/org/apache/spark/test/VectorQueryTest.scala
@@ -1,20 +1,21 @@
 /*
- * 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
+ * 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
+ *   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.
+ * 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 org.apache.spark.test
 
 import java.io.File

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/spark/spark-common/pom.xml
----------------------------------------------------------------------
diff --git a/spark/spark-common/pom.xml b/spark/spark-common/pom.xml
index 7697bd0..6325579 100644
--- a/spark/spark-common/pom.xml
+++ b/spark/spark-common/pom.xml
@@ -17,6 +17,7 @@
 		<PermGen>64m</PermGen>
 		<MaxPermGen>1024m</MaxPermGen>
 		<CodeCacheSize>512m</CodeCacheSize>
+		<main.basedir>${project.parent.basedir}</main.basedir>
 	</properties>
 
 	<dependencies>

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/spark/spark-common/src/main/java/hivemall/dataset/LogisticRegressionDataGeneratorUDTFWrapper.java
----------------------------------------------------------------------
diff --git a/spark/spark-common/src/main/java/hivemall/dataset/LogisticRegressionDataGeneratorUDTFWrapper.java b/spark/spark-common/src/main/java/hivemall/dataset/LogisticRegressionDataGeneratorUDTFWrapper.java
index 31b8419..c06e6bc 100644
--- a/spark/spark-common/src/main/java/hivemall/dataset/LogisticRegressionDataGeneratorUDTFWrapper.java
+++ b/spark/spark-common/src/main/java/hivemall/dataset/LogisticRegressionDataGeneratorUDTFWrapper.java
@@ -1,20 +1,21 @@
 /*
- * 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
+ * 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
+ *   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.
+ * 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.dataset;
 
 import java.lang.reflect.Field;

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/spark/spark-common/src/main/java/hivemall/ftvec/AddBiasUDFWrapper.java
----------------------------------------------------------------------
diff --git a/spark/spark-common/src/main/java/hivemall/ftvec/AddBiasUDFWrapper.java b/spark/spark-common/src/main/java/hivemall/ftvec/AddBiasUDFWrapper.java
index b85f8f0..b454fd9 100644
--- a/spark/spark-common/src/main/java/hivemall/ftvec/AddBiasUDFWrapper.java
+++ b/spark/spark-common/src/main/java/hivemall/ftvec/AddBiasUDFWrapper.java
@@ -1,20 +1,21 @@
 /*
- * 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
+ * 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
+ *   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.
+ * 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.ftvec;
 
 import java.util.Arrays;

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/spark/spark-common/src/main/java/hivemall/ftvec/AddFeatureIndexUDFWrapper.java
----------------------------------------------------------------------
diff --git a/spark/spark-common/src/main/java/hivemall/ftvec/AddFeatureIndexUDFWrapper.java b/spark/spark-common/src/main/java/hivemall/ftvec/AddFeatureIndexUDFWrapper.java
index 794281b..cba8e0f 100644
--- a/spark/spark-common/src/main/java/hivemall/ftvec/AddFeatureIndexUDFWrapper.java
+++ b/spark/spark-common/src/main/java/hivemall/ftvec/AddFeatureIndexUDFWrapper.java
@@ -1,20 +1,21 @@
 /*
- * 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
+ * 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
+ *   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.
+ * 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.ftvec;
 
 import java.util.Arrays;

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/spark/spark-common/src/main/java/hivemall/ftvec/ExtractFeatureUDFWrapper.java
----------------------------------------------------------------------
diff --git a/spark/spark-common/src/main/java/hivemall/ftvec/ExtractFeatureUDFWrapper.java b/spark/spark-common/src/main/java/hivemall/ftvec/ExtractFeatureUDFWrapper.java
index 9594a48..5924468 100644
--- a/spark/spark-common/src/main/java/hivemall/ftvec/ExtractFeatureUDFWrapper.java
+++ b/spark/spark-common/src/main/java/hivemall/ftvec/ExtractFeatureUDFWrapper.java
@@ -1,20 +1,21 @@
 /*
- * 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
+ * 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
+ *   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.
+ * 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.ftvec;
 
 import java.util.Arrays;

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/spark/spark-common/src/main/java/hivemall/ftvec/ExtractWeightUDFWrapper.java
----------------------------------------------------------------------
diff --git a/spark/spark-common/src/main/java/hivemall/ftvec/ExtractWeightUDFWrapper.java b/spark/spark-common/src/main/java/hivemall/ftvec/ExtractWeightUDFWrapper.java
index 11dde97..8580247 100644
--- a/spark/spark-common/src/main/java/hivemall/ftvec/ExtractWeightUDFWrapper.java
+++ b/spark/spark-common/src/main/java/hivemall/ftvec/ExtractWeightUDFWrapper.java
@@ -1,20 +1,21 @@
 /*
- * 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
+ * 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
+ *   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.
+ * 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.ftvec;
 
 import java.util.Arrays;

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/spark/spark-common/src/main/java/hivemall/ftvec/SortByFeatureUDFWrapper.java
----------------------------------------------------------------------
diff --git a/spark/spark-common/src/main/java/hivemall/ftvec/SortByFeatureUDFWrapper.java b/spark/spark-common/src/main/java/hivemall/ftvec/SortByFeatureUDFWrapper.java
index f7c17d3..5cef504 100644
--- a/spark/spark-common/src/main/java/hivemall/ftvec/SortByFeatureUDFWrapper.java
+++ b/spark/spark-common/src/main/java/hivemall/ftvec/SortByFeatureUDFWrapper.java
@@ -1,20 +1,21 @@
 /*
- * 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
+ * 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
+ *   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.
+ * 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.ftvec;
 
 import java.util.Arrays;

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/spark/spark-common/src/main/java/hivemall/ftvec/scaling/L2NormalizationUDFWrapper.java
----------------------------------------------------------------------
diff --git a/spark/spark-common/src/main/java/hivemall/ftvec/scaling/L2NormalizationUDFWrapper.java b/spark/spark-common/src/main/java/hivemall/ftvec/scaling/L2NormalizationUDFWrapper.java
index 161a4d1..db533be 100644
--- a/spark/spark-common/src/main/java/hivemall/ftvec/scaling/L2NormalizationUDFWrapper.java
+++ b/spark/spark-common/src/main/java/hivemall/ftvec/scaling/L2NormalizationUDFWrapper.java
@@ -1,20 +1,21 @@
 /*
- * 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
+ * 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
+ *   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.
+ * 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.ftvec.scaling;
 
 import java.util.ArrayList;

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/spark/spark-common/src/main/java/hivemall/knn/lsh/MinHashesUDFWrapper.java
----------------------------------------------------------------------
diff --git a/spark/spark-common/src/main/java/hivemall/knn/lsh/MinHashesUDFWrapper.java b/spark/spark-common/src/main/java/hivemall/knn/lsh/MinHashesUDFWrapper.java
index ebf78a3..d3bcbe6 100644
--- a/spark/spark-common/src/main/java/hivemall/knn/lsh/MinHashesUDFWrapper.java
+++ b/spark/spark-common/src/main/java/hivemall/knn/lsh/MinHashesUDFWrapper.java
@@ -1,20 +1,21 @@
 /*
- * 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
+ * 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
+ *   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.
+ * 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.knn.lsh;
 
 import java.util.Arrays;

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/spark/spark-common/src/main/java/hivemall/tools/mapred/RowIdUDFWrapper.java
----------------------------------------------------------------------
diff --git a/spark/spark-common/src/main/java/hivemall/tools/mapred/RowIdUDFWrapper.java b/spark/spark-common/src/main/java/hivemall/tools/mapred/RowIdUDFWrapper.java
index bca8607..f386223 100644
--- a/spark/spark-common/src/main/java/hivemall/tools/mapred/RowIdUDFWrapper.java
+++ b/spark/spark-common/src/main/java/hivemall/tools/mapred/RowIdUDFWrapper.java
@@ -1,20 +1,21 @@
 /*
- * 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
+ * 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
+ *   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.
+ * 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.tools.mapred;
 
 import java.util.UUID;



[37/50] [abbrv] incubator-hivemall git commit: Update README.md

Posted by my...@apache.org.
Update README.md

Project: http://git-wip-us.apache.org/repos/asf/incubator-hivemall/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-hivemall/commit/190cb6e7
Tree: http://git-wip-us.apache.org/repos/asf/incubator-hivemall/tree/190cb6e7
Diff: http://git-wip-us.apache.org/repos/asf/incubator-hivemall/diff/190cb6e7

Branch: refs/heads/master
Commit: 190cb6e79859bd3941883e9ea290a8db59d9f968
Parents: 2890954
Author: Makoto YUI <yu...@gmail.com>
Authored: Sat Nov 12 15:40:22 2016 +0900
Committer: GitHub <no...@github.com>
Committed: Sat Nov 12 15:40:22 2016 +0900

----------------------------------------------------------------------
 README.md | 153 +++------------------------------------------------------
 1 file changed, 6 insertions(+), 147 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/190cb6e7/README.md
----------------------------------------------------------------------
diff --git a/README.md b/README.md
index 79333aa..1aa9fea 100644
--- a/README.md
+++ b/README.md
@@ -25,163 +25,22 @@ Apache Hivemall: Hive scalable machine learning library
 [![License](http://img.shields.io/:license-Apache_v2-blue.svg)](https://github.com/myui/hivemall/blob/master/LICENSE)
 [![Coverage Status](https://coveralls.io/repos/github/myui/hivemall/badge.svg)](https://coveralls.io/github/myui/hivemall)
 
-**News:** Hivemall joins [Apache Incubator](http://hivemall.incubator.apache.org/)! :tada: Currently in the process of moving the project repository to ASF.
+**News:** Hivemall joins [Apache Incubator](http://hivemall.incubator.apache.org/)! :tada: Currently in the process of moving the project repository to ASF. This repository will be deprecated soon. Please move your stars and forks to [the new repository](https://github.com/apache/incubator-hivemall).
+
 [![Incubator](http://incubator.apache.org/images/egg-logo2.png "Apache Incubator")](http://hivemall.incubator.apache.org/)
 
-Hivemall is a scalable machine learning library that runs on Apache Hive.
-Hivemall is designed to be scalable to the number of training instances as well as the number of training features.
+Hivemall is a scalable machine learning library that runs on Apache Hive, Apache Spark, and Apache Pig. Hivemall is designed to be scalable to the number of training instances as well as the number of training features.
 
 ![logo](https://raw.github.com/myui/hivemall/master/resources/hivemall-logo-color-small.png "Hivemall's cute(!?) logo")
 
-Supported Algorithms
---------------------
-
-Hivemall provides machine learning functionality as well as feature engineering functions through UDFs/UDAFs/UDTFs of Hive. 
-
-## Binary Classification
-
-* [Perceptron](http://en.wikipedia.org/wiki/Perceptron)
-
-* Passive Aggressive (PA, PA1, PA2)
-
-* Confidence Weighted (CW)
-
-* Adaptive Regularization of Weight Vectors (AROW)
-
-* Soft Confidence Weighted (SCW1, SCW2)
-
-* AdaGradRDA (w/ hinge loss)
-
-* Factorization Machine (w/ logistic loss)
-
-_My recommendation is AROW, SCW1, AdaGradRDA, and Factorization Machine while it depends._
-
-## Multi-class Classification
-
-* [Perceptron](http://en.wikipedia.org/wiki/Perceptron)
-
-* Passive Aggressive (PA, PA1, PA2)
-
-* Confidence Weighted (CW)
-
-* Adaptive Regularization of Weight Vectors (AROW)
-
-* Soft Confidence Weighted (SCW1, SCW2)
-
-* Random Forest Classifier
-
-* Gradient Tree Boosting (_Experimental_)
-
-_My recommendation is AROW and SCW while it depends._
-
-## Regression
-
-* [Logistic Regression](http://en.wikipedia.org/wiki/Logistic_regression) using [Stochastic Gradient Descent](http://en.wikipedia.org/wiki/Stochastic_gradient_descent)
-
-* AdaGrad, AdaDelta (w/ logistic Loss)
-  
-* Passive Aggressive Regression (PA1, PA2)
-
-* AROW regression
-
-* Random Forest Regressor
-
-* Factorization Machine (w/ squared loss)
-
-* [Polynomial Regression](http://en.wikipedia.org/wiki/Polynomial_regression)
-
-_My recommendation for is AROW regression, AdaDelta, and Factorization Machine while it depends._
-
-## Recommendation
-
-* [Minhash](http://en.wikipedia.org/wiki/MinHash) ([LSH](http://en.wikipedia.org/wiki/Locality-sensitive_hashing) with jaccard index)
-
-* [Matrix Factorization](http://en.wikipedia.org/wiki/Matrix_decomposition) (sgd, adagrad)
-
-* Factorization Machine (squared loss for rating prediction)
-
-## k-Nearest Neighbor
-
-* [Minhash](http://en.wikipedia.org/wiki/MinHash) ([LSH](http://en.wikipedia.org/wiki/Locality-sensitive_hashing) with jaccard index)
-
-* b-Bit minhash
-
-* Brute-force search using Cosine similarity
-
-## Anomaly Detection
-
-* [Local Outlier Factor (LOF)](http://en.wikipedia.org/wiki/Local_outlier_factor)
-
-## Natural Language Processing
-
-* English/Japanese Text Tokenizer
-
-## Feature engineering
-  
-* [Feature Hashing](http://en.wikipedia.org/wiki/Feature_hashing) (MurmurHash, SHA1)
-
-* [Feature scaling](http://en.wikipedia.org/wiki/Feature_scaling) (Min-Max Normalization, Z-Score)
-
-* [Polynomial Features](http://en.wikipedia.org/wiki/Polynomial_kernel)
-
-* Feature instances amplifier that reduces iterations on training
-
-* [TF-IDF](http://en.wikipedia.org/wiki/Tf%E2%80%93idf) vectorizer
-
-* Bias clause
-
-* Data generator for one-vs-the-rest classifiers
-
-System requirements
---------------------
-
-* Hive 0.13 or later
-
-* Java 7 or later
-
-* Spark 1.6 or 2.0 for Hivemall on Spark
-
-* Pig 0.15 or later for Hivemall on Pig
-
 Basic Usage
 ------------
 
-[![Hivemall](https://gist.githubusercontent.com/myui/d29241262f9313dec706/raw/caead313efd829b42a4a4183285e8b53cf26ab62/hadoopsummit14_slideshare.png)](http://www.slideshare.net/myui/hivemall-hadoop-summit-2014-san-jose)
+[![Hivemall](https://gist.githubusercontent.com/myui/d29241262f9313dec706/raw/caead313efd829b42a4a4183285e8b53cf26ab62/hadoopsummit14_slideshare.png)](http://www.slideshare.net/myui/hadoopsummit16-myui)
 
-Find more examples on [our wiki page](https://github.com/myui/hivemall/wiki/) and find a brief introduction to Hivemall in [this slide](http://www.slideshare.net/myui/hivemall-hadoop-summit-2014-san-jose).
-
-Copyright
----------
-
-```
-Copyright (C) 2015-2016 Makoto YUI
-Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)  
-```
-
-Put the above copyrights for the services/softwares that use Hivemall.
+Find more examples on [our user guide](http://hivemall.incubator.apache.org/userguide/index.html) and find a brief introduction to Hivemall in [this slide](http://www.slideshare.net/myui/hadoopsummit16-myui).
 
 Support
 -------
 
-Support is through the [issue list](https://github.com/myui/hivemall/issues?q=label%3Aquestion), not by a direct e-mail. 
-
-References
-----------
-
-Please refer the following paper for research uses:
-
-* Makoto Yui. ``Hivemall: Scalable Machine Learning Library for Apache Hive'', [2014 Hadoop Summit](http://hadoopsummit.org/san-jose/), June 2014. \[[slide](http://www.slideshare.net/myui/hivemall-hadoop-summit-2014-san-jose)]
-
-* Makoto Yui and Isao Kojima. ``Hivemall: Hive scalable machine learning library'' (demo), [NIPS 2013 Workshop on Machine Learning Open Source Software: Towards Open Workflows](https://mloss.org/workshop/nips13/), Dec 2013.
-
-* Makoto Yui and Isao Kojima. ``A Database-Hadoop Hybrid Approach to Scalable Machine Learning'', Proc. IEEE 2nd International Congress on Big Data, July 2013 \[[paper](http://staff.aist.go.jp/m.yui/publications/bigdata2013myui.pdf)\] \[[slide](http://www.slideshare.net/myui/bigdata2013myui)\]
-
-Awards
-------
-
-* [InfoWorld Bossie Awards 2014: The best open source big data tools](http://www.infoworld.com/article/2688074/big-data/big-data-164727-bossie-awards-2014-the-best-open-source-big-data-tools.html)
-
-Acknowledgment
---------------
-
-This work was supported in part by a JSPS grant-in-aid for young scientists (B) #24700111 and a JSPS grant-in-aid for scientific research (A) #24240015.
+Support is through the [user@hivemall.incubator.apache.org](http://hivemall.incubator.apache.org/mail-lists.html), not by a direct e-mail. 


[14/50] [abbrv] incubator-hivemall git commit: Updated license headers

Posted by my...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/test/java/hivemall/ftvec/pairing/PolynomialFeaturesUDFTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/hivemall/ftvec/pairing/PolynomialFeaturesUDFTest.java b/core/src/test/java/hivemall/ftvec/pairing/PolynomialFeaturesUDFTest.java
index 9b98002..daf4e30 100644
--- a/core/src/test/java/hivemall/ftvec/pairing/PolynomialFeaturesUDFTest.java
+++ b/core/src/test/java/hivemall/ftvec/pairing/PolynomialFeaturesUDFTest.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.ftvec.pairing;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/test/java/hivemall/ftvec/pairing/PoweredFeaturesUDFTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/hivemall/ftvec/pairing/PoweredFeaturesUDFTest.java b/core/src/test/java/hivemall/ftvec/pairing/PoweredFeaturesUDFTest.java
index 72d23cd..25fcdb6 100644
--- a/core/src/test/java/hivemall/ftvec/pairing/PoweredFeaturesUDFTest.java
+++ b/core/src/test/java/hivemall/ftvec/pairing/PoweredFeaturesUDFTest.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.ftvec.pairing;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/test/java/hivemall/ftvec/scaling/L2NormalizationUDFTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/hivemall/ftvec/scaling/L2NormalizationUDFTest.java b/core/src/test/java/hivemall/ftvec/scaling/L2NormalizationUDFTest.java
index ea2af41..bd80577 100644
--- a/core/src/test/java/hivemall/ftvec/scaling/L2NormalizationUDFTest.java
+++ b/core/src/test/java/hivemall/ftvec/scaling/L2NormalizationUDFTest.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.ftvec.scaling;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/test/java/hivemall/ftvec/trans/TestBinarizeLabelUDTF.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/hivemall/ftvec/trans/TestBinarizeLabelUDTF.java b/core/src/test/java/hivemall/ftvec/trans/TestBinarizeLabelUDTF.java
index d93f7bf..a8d2beb 100644
--- a/core/src/test/java/hivemall/ftvec/trans/TestBinarizeLabelUDTF.java
+++ b/core/src/test/java/hivemall/ftvec/trans/TestBinarizeLabelUDTF.java
@@ -1,19 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.ftvec.trans;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/test/java/hivemall/ftvec/trans/VectorizeFeaturesUDFTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/hivemall/ftvec/trans/VectorizeFeaturesUDFTest.java b/core/src/test/java/hivemall/ftvec/trans/VectorizeFeaturesUDFTest.java
index 6dae9f4..e826963 100644
--- a/core/src/test/java/hivemall/ftvec/trans/VectorizeFeaturesUDFTest.java
+++ b/core/src/test/java/hivemall/ftvec/trans/VectorizeFeaturesUDFTest.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.ftvec.trans;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/test/java/hivemall/knn/distance/EuclidDistanceUDFTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/hivemall/knn/distance/EuclidDistanceUDFTest.java b/core/src/test/java/hivemall/knn/distance/EuclidDistanceUDFTest.java
index 71d2fa5..e91ec6d 100644
--- a/core/src/test/java/hivemall/knn/distance/EuclidDistanceUDFTest.java
+++ b/core/src/test/java/hivemall/knn/distance/EuclidDistanceUDFTest.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.knn.distance;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/test/java/hivemall/knn/lsh/MinHashUDFTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/hivemall/knn/lsh/MinHashUDFTest.java b/core/src/test/java/hivemall/knn/lsh/MinHashUDFTest.java
index 2f9351f..8a82efc 100644
--- a/core/src/test/java/hivemall/knn/lsh/MinHashUDFTest.java
+++ b/core/src/test/java/hivemall/knn/lsh/MinHashUDFTest.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.knn.lsh;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/test/java/hivemall/knn/lsh/bBitMinHashUDFTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/hivemall/knn/lsh/bBitMinHashUDFTest.java b/core/src/test/java/hivemall/knn/lsh/bBitMinHashUDFTest.java
index 4167752..14ef969 100644
--- a/core/src/test/java/hivemall/knn/lsh/bBitMinHashUDFTest.java
+++ b/core/src/test/java/hivemall/knn/lsh/bBitMinHashUDFTest.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.knn.lsh;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/test/java/hivemall/knn/similarity/CosineSimilarityUDFTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/hivemall/knn/similarity/CosineSimilarityUDFTest.java b/core/src/test/java/hivemall/knn/similarity/CosineSimilarityUDFTest.java
index 7a9c9a6..9825e9b 100644
--- a/core/src/test/java/hivemall/knn/similarity/CosineSimilarityUDFTest.java
+++ b/core/src/test/java/hivemall/knn/similarity/CosineSimilarityUDFTest.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.knn.similarity;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/test/java/hivemall/mf/BPRMatrixFactorizationUDTFTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/hivemall/mf/BPRMatrixFactorizationUDTFTest.java b/core/src/test/java/hivemall/mf/BPRMatrixFactorizationUDTFTest.java
index 8c13359..e28d318 100644
--- a/core/src/test/java/hivemall/mf/BPRMatrixFactorizationUDTFTest.java
+++ b/core/src/test/java/hivemall/mf/BPRMatrixFactorizationUDTFTest.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.mf;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/test/java/hivemall/mf/MatrixFactorizationAdaGradUDTFTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/hivemall/mf/MatrixFactorizationAdaGradUDTFTest.java b/core/src/test/java/hivemall/mf/MatrixFactorizationAdaGradUDTFTest.java
index 959aa8a..e587f01 100644
--- a/core/src/test/java/hivemall/mf/MatrixFactorizationAdaGradUDTFTest.java
+++ b/core/src/test/java/hivemall/mf/MatrixFactorizationAdaGradUDTFTest.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.mf;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/test/java/hivemall/mf/MatrixFactorizationSGDUDTFTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/hivemall/mf/MatrixFactorizationSGDUDTFTest.java b/core/src/test/java/hivemall/mf/MatrixFactorizationSGDUDTFTest.java
index 28560ed..53983c1 100644
--- a/core/src/test/java/hivemall/mf/MatrixFactorizationSGDUDTFTest.java
+++ b/core/src/test/java/hivemall/mf/MatrixFactorizationSGDUDTFTest.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.mf;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/test/java/hivemall/mix/client/MixRequestRouterTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/hivemall/mix/client/MixRequestRouterTest.java b/core/src/test/java/hivemall/mix/client/MixRequestRouterTest.java
index 607c52c..31301d8 100644
--- a/core/src/test/java/hivemall/mix/client/MixRequestRouterTest.java
+++ b/core/src/test/java/hivemall/mix/client/MixRequestRouterTest.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.mix.client;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/test/java/hivemall/model/FeatureValueTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/hivemall/model/FeatureValueTest.java b/core/src/test/java/hivemall/model/FeatureValueTest.java
index 2e289f5..2b6c832 100644
--- a/core/src/test/java/hivemall/model/FeatureValueTest.java
+++ b/core/src/test/java/hivemall/model/FeatureValueTest.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.model;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/test/java/hivemall/model/SpaceEfficientDenseModelTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/hivemall/model/SpaceEfficientDenseModelTest.java b/core/src/test/java/hivemall/model/SpaceEfficientDenseModelTest.java
index e3a1ed4..8106890 100644
--- a/core/src/test/java/hivemall/model/SpaceEfficientDenseModelTest.java
+++ b/core/src/test/java/hivemall/model/SpaceEfficientDenseModelTest.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.model;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/test/java/hivemall/regression/AdaGradUDTFTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/hivemall/regression/AdaGradUDTFTest.java b/core/src/test/java/hivemall/regression/AdaGradUDTFTest.java
index ce0161d..e7a0a89 100644
--- a/core/src/test/java/hivemall/regression/AdaGradUDTFTest.java
+++ b/core/src/test/java/hivemall/regression/AdaGradUDTFTest.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.regression;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/test/java/hivemall/smile/classification/DecisionTreeTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/hivemall/smile/classification/DecisionTreeTest.java b/core/src/test/java/hivemall/smile/classification/DecisionTreeTest.java
index 846b440..3c6116c 100644
--- a/core/src/test/java/hivemall/smile/classification/DecisionTreeTest.java
+++ b/core/src/test/java/hivemall/smile/classification/DecisionTreeTest.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (c) 2010 Haifeng Li
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.smile.classification;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/test/java/hivemall/smile/classification/RandomForestClassifierUDTFTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/hivemall/smile/classification/RandomForestClassifierUDTFTest.java b/core/src/test/java/hivemall/smile/classification/RandomForestClassifierUDTFTest.java
index 7cc966f..5f8518b 100644
--- a/core/src/test/java/hivemall/smile/classification/RandomForestClassifierUDTFTest.java
+++ b/core/src/test/java/hivemall/smile/classification/RandomForestClassifierUDTFTest.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.smile.classification;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/test/java/hivemall/smile/regression/RegressionTreeTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/hivemall/smile/regression/RegressionTreeTest.java b/core/src/test/java/hivemall/smile/regression/RegressionTreeTest.java
index 0c8e43c..20f44b3 100644
--- a/core/src/test/java/hivemall/smile/regression/RegressionTreeTest.java
+++ b/core/src/test/java/hivemall/smile/regression/RegressionTreeTest.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.smile.regression;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/test/java/hivemall/smile/tools/TreePredictUDFTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/hivemall/smile/tools/TreePredictUDFTest.java b/core/src/test/java/hivemall/smile/tools/TreePredictUDFTest.java
index 2823feb..504ea86 100644
--- a/core/src/test/java/hivemall/smile/tools/TreePredictUDFTest.java
+++ b/core/src/test/java/hivemall/smile/tools/TreePredictUDFTest.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (c) 2010 Haifeng Li
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.smile.tools;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/test/java/hivemall/smile/vm/StackMachineTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/hivemall/smile/vm/StackMachineTest.java b/core/src/test/java/hivemall/smile/vm/StackMachineTest.java
index e3a917e..4a2dcd8 100644
--- a/core/src/test/java/hivemall/smile/vm/StackMachineTest.java
+++ b/core/src/test/java/hivemall/smile/vm/StackMachineTest.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (c) 2010 Haifeng Li
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.smile.vm;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/test/java/hivemall/tools/array/ToStringArrayUDFTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/hivemall/tools/array/ToStringArrayUDFTest.java b/core/src/test/java/hivemall/tools/array/ToStringArrayUDFTest.java
index 2686cd7..132d5bc 100644
--- a/core/src/test/java/hivemall/tools/array/ToStringArrayUDFTest.java
+++ b/core/src/test/java/hivemall/tools/array/ToStringArrayUDFTest.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.tools.array;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/test/java/hivemall/utils/ArrayUtilsTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/hivemall/utils/ArrayUtilsTest.java b/core/src/test/java/hivemall/utils/ArrayUtilsTest.java
index d1a7399..8987fd8 100644
--- a/core/src/test/java/hivemall/utils/ArrayUtilsTest.java
+++ b/core/src/test/java/hivemall/utils/ArrayUtilsTest.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.utils;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/test/java/hivemall/utils/BitUtilsTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/hivemall/utils/BitUtilsTest.java b/core/src/test/java/hivemall/utils/BitUtilsTest.java
index 1ee2ad6..cab00ef 100644
--- a/core/src/test/java/hivemall/utils/BitUtilsTest.java
+++ b/core/src/test/java/hivemall/utils/BitUtilsTest.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.utils;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/test/java/hivemall/utils/buffer/HeapBufferTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/hivemall/utils/buffer/HeapBufferTest.java b/core/src/test/java/hivemall/utils/buffer/HeapBufferTest.java
index f7f3968..8389a8c 100644
--- a/core/src/test/java/hivemall/utils/buffer/HeapBufferTest.java
+++ b/core/src/test/java/hivemall/utils/buffer/HeapBufferTest.java
@@ -1,19 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.utils.buffer;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/test/java/hivemall/utils/codec/Base91Test.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/hivemall/utils/codec/Base91Test.java b/core/src/test/java/hivemall/utils/codec/Base91Test.java
index 3d89e63..809ba97 100644
--- a/core/src/test/java/hivemall/utils/codec/Base91Test.java
+++ b/core/src/test/java/hivemall/utils/codec/Base91Test.java
@@ -1,19 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.utils.codec;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/test/java/hivemall/utils/codec/DeflateCodecTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/hivemall/utils/codec/DeflateCodecTest.java b/core/src/test/java/hivemall/utils/codec/DeflateCodecTest.java
index 6387ce0..323fd38 100644
--- a/core/src/test/java/hivemall/utils/codec/DeflateCodecTest.java
+++ b/core/src/test/java/hivemall/utils/codec/DeflateCodecTest.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.utils.codec;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/test/java/hivemall/utils/codec/ZigZagLEB128CodecTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/hivemall/utils/codec/ZigZagLEB128CodecTest.java b/core/src/test/java/hivemall/utils/codec/ZigZagLEB128CodecTest.java
index 3f34e54..caeed84 100644
--- a/core/src/test/java/hivemall/utils/codec/ZigZagLEB128CodecTest.java
+++ b/core/src/test/java/hivemall/utils/codec/ZigZagLEB128CodecTest.java
@@ -1,3 +1,21 @@
+/*
+ * 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.utils.codec;
 
 import hivemall.utils.io.FastByteArrayOutputStream;

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/test/java/hivemall/utils/collections/DoubleArray3DTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/hivemall/utils/collections/DoubleArray3DTest.java b/core/src/test/java/hivemall/utils/collections/DoubleArray3DTest.java
index e7142df..177a345 100644
--- a/core/src/test/java/hivemall/utils/collections/DoubleArray3DTest.java
+++ b/core/src/test/java/hivemall/utils/collections/DoubleArray3DTest.java
@@ -1,19 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.utils.collections;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/test/java/hivemall/utils/collections/DoubleRingBufferTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/hivemall/utils/collections/DoubleRingBufferTest.java b/core/src/test/java/hivemall/utils/collections/DoubleRingBufferTest.java
index 088fded..c07a8af 100644
--- a/core/src/test/java/hivemall/utils/collections/DoubleRingBufferTest.java
+++ b/core/src/test/java/hivemall/utils/collections/DoubleRingBufferTest.java
@@ -1,19 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.utils.collections;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/test/java/hivemall/utils/collections/Int2FloatOpenHashMapTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/hivemall/utils/collections/Int2FloatOpenHashMapTest.java b/core/src/test/java/hivemall/utils/collections/Int2FloatOpenHashMapTest.java
index 1b3e6fa..14635be 100644
--- a/core/src/test/java/hivemall/utils/collections/Int2FloatOpenHashMapTest.java
+++ b/core/src/test/java/hivemall/utils/collections/Int2FloatOpenHashMapTest.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.utils.collections;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/test/java/hivemall/utils/collections/Int2LongOpenHashMapTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/hivemall/utils/collections/Int2LongOpenHashMapTest.java b/core/src/test/java/hivemall/utils/collections/Int2LongOpenHashMapTest.java
index 10971bb..1186bdf 100644
--- a/core/src/test/java/hivemall/utils/collections/Int2LongOpenHashMapTest.java
+++ b/core/src/test/java/hivemall/utils/collections/Int2LongOpenHashMapTest.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.utils.collections;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/test/java/hivemall/utils/collections/IntOpenHashMapTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/hivemall/utils/collections/IntOpenHashMapTest.java b/core/src/test/java/hivemall/utils/collections/IntOpenHashMapTest.java
index e904597..29a5a81 100644
--- a/core/src/test/java/hivemall/utils/collections/IntOpenHashMapTest.java
+++ b/core/src/test/java/hivemall/utils/collections/IntOpenHashMapTest.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.utils.collections;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/test/java/hivemall/utils/collections/IntOpenHashTableTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/hivemall/utils/collections/IntOpenHashTableTest.java b/core/src/test/java/hivemall/utils/collections/IntOpenHashTableTest.java
index 0dac7b8..3babb3d 100644
--- a/core/src/test/java/hivemall/utils/collections/IntOpenHashTableTest.java
+++ b/core/src/test/java/hivemall/utils/collections/IntOpenHashTableTest.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.utils.collections;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/test/java/hivemall/utils/collections/OpenHashMapTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/hivemall/utils/collections/OpenHashMapTest.java b/core/src/test/java/hivemall/utils/collections/OpenHashMapTest.java
index 03ba1db..e3cc018 100644
--- a/core/src/test/java/hivemall/utils/collections/OpenHashMapTest.java
+++ b/core/src/test/java/hivemall/utils/collections/OpenHashMapTest.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.utils.collections;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/test/java/hivemall/utils/collections/OpenHashTableTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/hivemall/utils/collections/OpenHashTableTest.java b/core/src/test/java/hivemall/utils/collections/OpenHashTableTest.java
index 97c2917..d5a465c 100644
--- a/core/src/test/java/hivemall/utils/collections/OpenHashTableTest.java
+++ b/core/src/test/java/hivemall/utils/collections/OpenHashTableTest.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.utils.collections;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/test/java/hivemall/utils/collections/SparseIntArrayTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/hivemall/utils/collections/SparseIntArrayTest.java b/core/src/test/java/hivemall/utils/collections/SparseIntArrayTest.java
index 4783c9b..68d0f6d 100644
--- a/core/src/test/java/hivemall/utils/collections/SparseIntArrayTest.java
+++ b/core/src/test/java/hivemall/utils/collections/SparseIntArrayTest.java
@@ -1,3 +1,21 @@
+/*
+ * 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.utils.collections;
 
 import java.util.Random;

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/test/java/hivemall/utils/hadoop/HadoopUtilsTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/hivemall/utils/hadoop/HadoopUtilsTest.java b/core/src/test/java/hivemall/utils/hadoop/HadoopUtilsTest.java
index 1a580d3..cd8dcc6 100644
--- a/core/src/test/java/hivemall/utils/hadoop/HadoopUtilsTest.java
+++ b/core/src/test/java/hivemall/utils/hadoop/HadoopUtilsTest.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.utils.hadoop;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/test/java/hivemall/utils/hadoop/Text2Test.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/hivemall/utils/hadoop/Text2Test.java b/core/src/test/java/hivemall/utils/hadoop/Text2Test.java
index 055d1c8..48e6a21 100644
--- a/core/src/test/java/hivemall/utils/hadoop/Text2Test.java
+++ b/core/src/test/java/hivemall/utils/hadoop/Text2Test.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.utils.hadoop;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/test/java/hivemall/utils/hashing/MurmurHash3Test.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/hivemall/utils/hashing/MurmurHash3Test.java b/core/src/test/java/hivemall/utils/hashing/MurmurHash3Test.java
index 7eb4da5..dd47a55 100644
--- a/core/src/test/java/hivemall/utils/hashing/MurmurHash3Test.java
+++ b/core/src/test/java/hivemall/utils/hashing/MurmurHash3Test.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.utils.hashing;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/test/java/hivemall/utils/io/Base91OutputStreamTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/hivemall/utils/io/Base91OutputStreamTest.java b/core/src/test/java/hivemall/utils/io/Base91OutputStreamTest.java
index 741702a..99a132f 100644
--- a/core/src/test/java/hivemall/utils/io/Base91OutputStreamTest.java
+++ b/core/src/test/java/hivemall/utils/io/Base91OutputStreamTest.java
@@ -1,19 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.utils.io;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/test/java/hivemall/utils/lang/HalfFloatTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/hivemall/utils/lang/HalfFloatTest.java b/core/src/test/java/hivemall/utils/lang/HalfFloatTest.java
index 9332a2b..42396c2 100644
--- a/core/src/test/java/hivemall/utils/lang/HalfFloatTest.java
+++ b/core/src/test/java/hivemall/utils/lang/HalfFloatTest.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.utils.lang;
 



[24/50] [abbrv] incubator-hivemall git commit: Updated license headers of *.sh

Posted by my...@apache.org.
Updated license headers of *.sh

Project: http://git-wip-us.apache.org/repos/asf/incubator-hivemall/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-hivemall/commit/24f1779c
Tree: http://git-wip-us.apache.org/repos/asf/incubator-hivemall/tree/24f1779c
Diff: http://git-wip-us.apache.org/repos/asf/incubator-hivemall/diff/24f1779c

Branch: refs/heads/master
Commit: 24f1779c240ac9f92d29268d19d1d424700bd6c8
Parents: d806bf4
Author: myui <yu...@gmail.com>
Authored: Mon Nov 7 13:59:11 2016 +0900
Committer: myui <yu...@gmail.com>
Committed: Mon Nov 7 13:59:11 2016 +0900

----------------------------------------------------------------------
 bin/build.sh                 | 18 ++++++++++++++++++
 bin/build_site.sh            | 18 ++++++++++++++++++
 bin/build_xgboost.sh         | 27 ++++++++++++++-------------
 bin/format_header.sh         | 18 ++++++++++++++++++
 bin/maven_central_release.sh | 18 ++++++++++++++++++
 bin/mixserv_cluster.sh       | 27 ++++++++++++++-------------
 bin/mixserv_daemon.sh        | 27 ++++++++++++++-------------
 bin/start_mixserv.sh         | 27 ++++++++++++++-------------
 bin/stop_mixserv.sh          | 27 ++++++++++++++-------------
 pom.xml                      |  1 +
 10 files changed, 143 insertions(+), 65 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/24f1779c/bin/build.sh
----------------------------------------------------------------------
diff --git a/bin/build.sh b/bin/build.sh
index 32fecd0..9c39835 100755
--- a/bin/build.sh
+++ b/bin/build.sh
@@ -1,4 +1,22 @@
 #!/bin/sh
+#
+# 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.
+#
 
 if [ "$HIVEMALL_HOME" == "" ]; then
   if [ -e ../bin/${0##*/} ]; then

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/24f1779c/bin/build_site.sh
----------------------------------------------------------------------
diff --git a/bin/build_site.sh b/bin/build_site.sh
index bad6548..04496b4 100755
--- a/bin/build_site.sh
+++ b/bin/build_site.sh
@@ -1,4 +1,22 @@
 #!/bin/sh
+#
+# 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.
+#
 
 if [ "$HIVEMALL_HOME" == "" ]; then
     if [ -e ../bin/${0##*/} ]; then

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/24f1779c/bin/build_xgboost.sh
----------------------------------------------------------------------
diff --git a/bin/build_xgboost.sh b/bin/build_xgboost.sh
index 7b97eea..7dcd4f7 100755
--- a/bin/build_xgboost.sh
+++ b/bin/build_xgboost.sh
@@ -1,21 +1,22 @@
 #!/bin/bash
-
-# Hivemall: Hive scalable Machine Learning Library
 #
-# Copyright (C) 2015 Makoto YUI
-# Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+# 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
 #
-# Licensed 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
 #
-#         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.
 #
-# 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.
 
 set -eu
 set -o pipefail

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/24f1779c/bin/format_header.sh
----------------------------------------------------------------------
diff --git a/bin/format_header.sh b/bin/format_header.sh
index e02b1b5..349aa49 100755
--- a/bin/format_header.sh
+++ b/bin/format_header.sh
@@ -1,4 +1,22 @@
 #!/bin/sh
+#
+# 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.
+#
 
 if [ "$HIVEMALL_HOME" == "" ]; then
   if [ -e ../bin/${0##*/} ]; then

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/24f1779c/bin/maven_central_release.sh
----------------------------------------------------------------------
diff --git a/bin/maven_central_release.sh b/bin/maven_central_release.sh
index fbcbcca..0137a88 100755
--- a/bin/maven_central_release.sh
+++ b/bin/maven_central_release.sh
@@ -1,4 +1,22 @@
 #!/bin/sh
+#
+# 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.
+#
 
 if [ "$HIVEMALL_HOME" == "" ]; then
   if [ -e ../bin/${0##*/} ]; then

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/24f1779c/bin/mixserv_cluster.sh
----------------------------------------------------------------------
diff --git a/bin/mixserv_cluster.sh b/bin/mixserv_cluster.sh
index ac31b90..83707d7 100755
--- a/bin/mixserv_cluster.sh
+++ b/bin/mixserv_cluster.sh
@@ -1,21 +1,22 @@
 #!/bin/sh
-
-# Hivemall: Hive scalable Machine Learning Library
 #
-# Copyright (C) 2015 Makoto YUI
-# Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+# 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
 #
-# Licensed 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
 #
-#         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.
 #
-# 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.
 
 # Start MIX server instances on each machine specified
 # in the conf/MIXSERV_LIST file

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/24f1779c/bin/mixserv_daemon.sh
----------------------------------------------------------------------
diff --git a/bin/mixserv_daemon.sh b/bin/mixserv_daemon.sh
index eb2887f..f7af4d9 100755
--- a/bin/mixserv_daemon.sh
+++ b/bin/mixserv_daemon.sh
@@ -1,21 +1,22 @@
 #!/bin/sh
-
-# Hivemall: Hive scalable Machine Learning Library
 #
-# Copyright (C) 2015 Makoto YUI
-# Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+# 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
 #
-# Licensed 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
 #
-#         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.
 #
-# 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.
 
 usage="Usage: mixserv_daemon.sh (start|stop|status)"
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/24f1779c/bin/start_mixserv.sh
----------------------------------------------------------------------
diff --git a/bin/start_mixserv.sh b/bin/start_mixserv.sh
index b3faee1..6de0c29 100755
--- a/bin/start_mixserv.sh
+++ b/bin/start_mixserv.sh
@@ -1,21 +1,22 @@
 #!/bin/sh
-
-# Hivemall: Hive scalable Machine Learning Library
 #
-# Copyright (C) 2015 Makoto YUI
-# Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+# 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
 #
-# Licensed 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
 #
-#         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.
 #
-# 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.
 
 if [ "$HIVEMALL_HOME" == "" ]; then
   if [ -e ../bin/${0##*/} ]; then

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/24f1779c/bin/stop_mixserv.sh
----------------------------------------------------------------------
diff --git a/bin/stop_mixserv.sh b/bin/stop_mixserv.sh
index a22fd6e..ef860c0 100755
--- a/bin/stop_mixserv.sh
+++ b/bin/stop_mixserv.sh
@@ -1,21 +1,22 @@
 #!/bin/sh
-
-# Hivemall: Hive scalable Machine Learning Library
 #
-# Copyright (C) 2015 Makoto YUI
-# Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+# 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
 #
-# Licensed 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
 #
-#         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.
 #
-# 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.
 
 if [ "$HIVEMALL_HOME" == "" ]; then
   if [ -e ../bin/${0##*/} ]; then

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/24f1779c/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index dcc0917..f36a36c 100644
--- a/pom.xml
+++ b/pom.xml
@@ -396,6 +396,7 @@
 							<include>src/test/**/*.java</include>
 							<include>src/main/**/*.scala</include>
 							<include>src/test/**/*.scala</include>
+							<include>bin/*.sh</include>
 						</includes>
 						<encoding>UTF-8</encoding>
 						<headerDefinitions>


[43/50] [abbrv] incubator-hivemall git commit: Add usage of each_top_k

Posted by my...@apache.org.
Add usage of each_top_k

Project: http://git-wip-us.apache.org/repos/asf/incubator-hivemall/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-hivemall/commit/ae2307f2
Tree: http://git-wip-us.apache.org/repos/asf/incubator-hivemall/tree/ae2307f2
Diff: http://git-wip-us.apache.org/repos/asf/incubator-hivemall/diff/ae2307f2

Branch: refs/heads/master
Commit: ae2307f260a637714e12c30a4a2c4b5b4afa584e
Parents: e987711
Author: myui <yu...@gmail.com>
Authored: Wed Nov 16 17:40:11 2016 +0900
Committer: myui <yu...@gmail.com>
Committed: Wed Nov 16 17:40:11 2016 +0900

----------------------------------------------------------------------
 docs/gitbook/misc/topk.md | 185 ++++++++++++++++++++++++++++-------------
 1 file changed, 127 insertions(+), 58 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/ae2307f2/docs/gitbook/misc/topk.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/misc/topk.md b/docs/gitbook/misc/topk.md
index 3d072ed..d6e7b93 100644
--- a/docs/gitbook/misc/topk.md
+++ b/docs/gitbook/misc/topk.md
@@ -21,17 +21,86 @@
 
 This function is particularly useful for applying a similarity/distance function where the computation complexity is **O(nm)**.
 
-`each_top_k` is very fast when compared to other methods running top-k queries (e.g., [`rank/distributed by`](https://ragrawal.wordpress.com/2011/11/18/extract-top-n-records-in-each-group-in-hadoophive/)) in Hive.
+`each_top_k` is very fast when compared to other methods running top-k queries (e.g., [`rank/distribute by`](https://ragrawal.wordpress.com/2011/11/18/extract-top-n-records-in-each-group-in-hadoophive/)) in Hive.
 
 ## Caution
 * `each_top_k` is supported from Hivemall v0.3.2-3 or later.
-* This UDTF assumes that input records are sorted by `group`. Use `DISTRIBUTED BY group SORTED BY group` to ensure that. Or, you can use `LEFT OUTER JOIN` for certain cases.
+* This UDTF assumes that input records are sorted by `group`. Use `DISTRIBUTE BY group SORT BY group` to ensure that. Or, you can use `LEFT OUTER JOIN` for certain cases.
 * It takes variable lengths arguments in `argN`. 
 * The third argument `value` is used for the comparison.
 * `Any number types` or `timestamp` are accepted for the type of `value`.
 * If k is less than 0, reverse order is used and `tail-K` records are returned for each `group`.
 * Note that this function returns [a pseudo ranking](http://www.michaelpollmeier.com/selecting-top-k-items-from-a-list-efficiently-in-java-groovy/) for top-k. It always returns `at-most K` records for each group. The ranking scheme is similar to `dense_rank` but slightly different in certain cases.
 
+# Efficient Top-k Query Processing using `each_top_k`
+
+Efficient processing of Top-k queries is a crucial requirement in many interactive environments that involve massive amounts of data. 
+Our Hive extension `each_top_k` helps running Top-k processing efficiently.
+
+- Suppose the following table as the input
+
+|student | class | score |
+|:------:|:-----:|:-----:|
+|1       | b     | 70    |
+|2       | a     | 80    |
+|3       | a     | 90    |
+|4       | b     | 50    |
+|5       | a     | 70    |
+|6       | b     | 60    |
+
+- Then, list top-2 students for each class
+
+|student | class | score | rank |
+|:------:|:-----:|:-----:|:----:|
+|3       | a     | 90    | 1    |
+|2       | a     | 80    | 2    |
+|1       | b     | 70    | 1    |
+|6       | b     | 60    | 2    |
+
+The standard way using SQL window function would be as follows:
+
+```sql
+SELECT 
+  student, class, score, rank
+FROM (
+  SELECT
+    student, class, score, 
+    rank() over (PARTITION BY class ORDER BY score DESC) as rank
+  FROM
+    table
+) t
+WHRE rank <= 2
+```
+
+An alternative and efficient way to compute top-k items using `each_top_k` is as follows:
+
+```sql
+SELECT 
+  each_top_k(
+    2, class, score,
+    class, student -- output columns other in addition to rank and score
+  ) as (rank, score, class, student)
+FROM (
+  SELECT * FROM table
+  CLUSTER BY class -- Mandatory for `each_top_k`
+) t
+```
+
+> #### Note
+`CLUSTER BY x` is a synonym of `DISTRIBUTE BY x CLASS SORT BY x` and required when using `each_top_k`.
+
+The function signature of `each_top_k` is `each_top_k(int k, ANY group, double value, arg1, arg2, ..., argN)` and it returns a relation `(int rank, double value, arg1, arg2, .., argN)`.
+
+Any number types or timestamp are accepted for the type of `value` but it MUST be not NULL. 
+Do null hanlding like `if(value is null, -1, value)` to avoid null.
+
+If `k` is less than 0, reverse order is used and tail-K records are returned for each `group`.
+
+The ranking semantics of `each_top_k` follows SQL's `dense_rank` and then limits results by `k`. 
+
+> #### Caution
+`each_top_k` is benefical where the number of grouping keys are large. If the number of grouping keys are not so large (e.g., less than 100), consider using `rank() over` instead.
+
 # Usage
 
 ## top-k clicks 
@@ -77,34 +146,34 @@ FROM
   LEFT OUTER JOIN train_hivemall t1;
 ```
 
-```
-1       0.8594650626182556      12      10514   0
-2       0.8585299849510193      12      11719   0
-3       0.856602132320404       12      21009   0
-4       0.8562054634094238      12      17582   0
-5       0.8516314029693604      12      22006   0
-6       0.8499397039413452      12      25364   0
-7       0.8467264771461487      12      900     0
-8       0.8463355302810669      12      8018    0
-9       0.8439178466796875      12      7041    0
-10      0.8438876867294312      12      21595   0
-1       0.8390793800354004      25      21125   0
-2       0.8344510793685913      25      14073   0
-3       0.8340602517127991      25      9008    0
-4       0.8328862190246582      25      6598    0
-5       0.8301891088485718      25      943     0
-6       0.8271955251693726      25      20400   0
-7       0.8255619406700134      25      10922   0
-8       0.8241575956344604      25      8477    0
-9       0.822281539440155       25      25977   0
-10      0.8205751180648804      25      21115   0
-1       0.9761330485343933      34      2513    0
-2       0.9536819458007812      34      8697    0
-3       0.9531533122062683      34      7326    0
-4       0.9493276476860046      34      15173   0
-5       0.9480557441711426      34      19468   0
-...
-```
+| rank | similarity | base_id | neighbor_id | y |
+|:----:|:----------:|:-------:|:-----------:|:-:|
+| 1  | 0.8594650626182556 | 12 | 10514 | 0 |
+| 2  | 0.8585299849510193 | 12 | 11719 | 0 |
+| 3  | 0.856602132320404  | 12 | 21009 | 0 |
+| 4  | 0.8562054634094238 | 12 | 17582 | 0 |
+| 5  | 0.8516314029693604 | 12 | 22006 | 0 |
+| 6  | 0.8499397039413452 | 12 | 25364 | 0 |
+| 7  | 0.8467264771461487 | 12 | 900   | 0 |
+| 8  | 0.8463355302810669 | 12 | 8018  | 0 |
+| 9  | 0.8439178466796875 | 12 | 7041  | 0 |
+| 10 | 0.8438876867294312 | 12 | 21595 | 0 |
+| 1  | 0.8390793800354004 | 25 | 21125 | 0 |
+| 2  | 0.8344510793685913 | 25 | 14073 | 0 |
+| 3  | 0.8340602517127991 | 25 | 9008  | 0 |
+| 4  | 0.8328862190246582 | 25 | 6598  | 0 |
+| 5  | 0.8301891088485718 | 25 | 943   | 0 |
+| 6  | 0.8271955251693726 | 25 | 20400 | 0 |
+| 7  | 0.8255619406700134 | 25 | 10922 | 0 |
+| 8  | 0.8241575956344604 | 25 | 8477  | 0 |
+| 9  | 0.822281539440155  | 25 | 25977 | 0 |
+| 10 | 0.8205751180648804 | 25 | 21115 | 0 |
+| 1  | 0.9761330485343933 | 34 | 2513  | 0 |
+| 2  | 0.9536819458007812 | 34 | 8697  | 0 |
+| 3  | 0.9531533122062683 | 34 | 7326  | 0 |
+| 4  | 0.9493276476860046 | 34 | 15173 | 0 |
+| 5  | 0.9480557441711426 | 34 | 19468 | 0 |
+| .. | .. | .. | .. | .. |
 
 ### Explicit grouping using `distribute by` and `sort by`
 
@@ -277,31 +346,31 @@ FROM
 -- limit 25
 ```
 
-```
-1       0.4383084177970886      1       7503    0
-2       0.44166821241378784     1       10143   0
-3       0.4424300789833069      1       11073   0
-4       0.44254064559936523     1       17782   0
-5       0.4442034363746643      1       18556   0
-6       0.45163780450820923     1       3786    0
-7       0.45244503021240234     1       10242   0
-8       0.4525672197341919      1       21657   0
-9       0.4527127146720886      1       17218   0
-10      0.45314133167266846     1       25141   0
-1       0.44030147790908813     2       3786    0
-2       0.4408798813819885      2       23386   0
-3       0.44112563133239746     2       11073   0
-4       0.4415401816368103      2       22853   0
-5       0.4422193765640259      2       21657   0
-6       0.4429032802581787      2       10143   0
-7       0.4435907006263733      2       24413   0
-8       0.44569307565689087     2       7503    0
-9       0.4460843801498413      2       25141   0
-10      0.4464914798736572      2       24289   0
-1       0.43862903118133545     3       23150   1
-2       0.4398220181465149      3       9881    1
-3       0.44283604621887207     3       27121   0
-4       0.4432108402252197      3       26220   1
-5       0.44323229789733887     3       18541   0
-...
-```
\ No newline at end of file
+| rank | similarity | base_id | neighbor_id | y |
+|:----:|:----------:|:-------:|:-----------:|:-:|
+| 1  | 0.4383084177970886  | 1 | 7503  | 0 |
+| 2  | 0.44166821241378784 | 1 | 10143 | 0 |
+| 3  | 0.4424300789833069  | 1 | 11073 | 0 |
+| 4  | 0.44254064559936523 | 1 | 17782 | 0 |
+| 5  | 0.4442034363746643  | 1 | 18556 | 0 |
+| 6  | 0.45163780450820923 | 1 | 3786  | 0 |
+| 7  | 0.45244503021240234 | 1 | 10242 | 0 |
+| 8  | 0.4525672197341919  | 1 | 21657 | 0 |
+| 9  | 0.4527127146720886  | 1 | 17218 | 0 |
+| 10 | 0.45314133167266846 | 1 | 25141 | 0 |
+| 1  | 0.44030147790908813 | 2 | 3786  | 0 |
+| 2  | 0.4408798813819885  | 2 | 23386 | 0 |
+| 3  | 0.44112563133239746 | 2 | 11073 | 0 |
+| 4  | 0.4415401816368103  | 2 | 22853 | 0 |
+| 5  | 0.4422193765640259  | 2 | 21657 | 0 |
+| 6  | 0.4429032802581787  | 2 | 10143 | 0 |
+| 7  | 0.4435907006263733  | 2 | 24413 | 0 |
+| 8  | 0.44569307565689087 | 2 | 7503  | 0 |
+| 9  | 0.4460843801498413  | 2 | 25141 | 0 |
+| 10 | 0.4464914798736572  | 2 | 24289 | 0 |
+| 1  | 0.43862903118133545 | 3 | 23150 | 1 |
+| 2  | 0.4398220181465149  | 3 | 9881  | 1 |
+| 3  | 0.44283604621887207 | 3 | 27121 | 0 |
+| 4  | 0.4432108402252197  | 3 | 26220 | 1 |
+| 5  | 0.44323229789733887 | 3 | 18541 | 0 |
+| .. | .. | .. | .. | .. |


[33/50] [abbrv] incubator-hivemall git commit: Removed a duplicate license header

Posted by my...@apache.org.
Removed a duplicate license header

Project: http://git-wip-us.apache.org/repos/asf/incubator-hivemall/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-hivemall/commit/2f8e44a3
Tree: http://git-wip-us.apache.org/repos/asf/incubator-hivemall/tree/2f8e44a3
Diff: http://git-wip-us.apache.org/repos/asf/incubator-hivemall/diff/2f8e44a3

Branch: refs/heads/master
Commit: 2f8e44a3f87e517fb49b110c7e70b0630af93e3b
Parents: 8cef8a3
Author: myui <yu...@gmail.com>
Authored: Sat Nov 12 14:53:45 2016 +0900
Committer: myui <yu...@gmail.com>
Committed: Sat Nov 12 14:53:45 2016 +0900

----------------------------------------------------------------------
 .../hivemall/utils/buffer/DynamicByteArray.java    | 17 -----------------
 1 file changed, 17 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/2f8e44a3/core/src/main/java/hivemall/utils/buffer/DynamicByteArray.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/utils/buffer/DynamicByteArray.java b/core/src/main/java/hivemall/utils/buffer/DynamicByteArray.java
index 87a8fbb..2d91870 100644
--- a/core/src/main/java/hivemall/utils/buffer/DynamicByteArray.java
+++ b/core/src/main/java/hivemall/utils/buffer/DynamicByteArray.java
@@ -16,23 +16,6 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-/*
- * 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.utils.buffer;
 
 import java.io.IOException;


[15/50] [abbrv] incubator-hivemall git commit: Updated license headers

Posted by my...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/utils/lang/HalfFloat.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/utils/lang/HalfFloat.java b/core/src/main/java/hivemall/utils/lang/HalfFloat.java
index d394fd8..7c778c4 100644
--- a/core/src/main/java/hivemall/utils/lang/HalfFloat.java
+++ b/core/src/main/java/hivemall/utils/lang/HalfFloat.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.utils.lang;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/utils/lang/Identifier.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/utils/lang/Identifier.java b/core/src/main/java/hivemall/utils/lang/Identifier.java
index b3cf63d..70ea80d 100644
--- a/core/src/main/java/hivemall/utils/lang/Identifier.java
+++ b/core/src/main/java/hivemall/utils/lang/Identifier.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.utils.lang;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/utils/lang/NumberUtils.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/utils/lang/NumberUtils.java b/core/src/main/java/hivemall/utils/lang/NumberUtils.java
index 2e273bb..0d3f895 100644
--- a/core/src/main/java/hivemall/utils/lang/NumberUtils.java
+++ b/core/src/main/java/hivemall/utils/lang/NumberUtils.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.utils.lang;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/utils/lang/ObjectUtils.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/utils/lang/ObjectUtils.java b/core/src/main/java/hivemall/utils/lang/ObjectUtils.java
index 7a1a08b..fd4af32 100644
--- a/core/src/main/java/hivemall/utils/lang/ObjectUtils.java
+++ b/core/src/main/java/hivemall/utils/lang/ObjectUtils.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.utils.lang;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/utils/lang/Preconditions.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/utils/lang/Preconditions.java b/core/src/main/java/hivemall/utils/lang/Preconditions.java
index 9dd8be0..4fa2bdd 100644
--- a/core/src/main/java/hivemall/utils/lang/Preconditions.java
+++ b/core/src/main/java/hivemall/utils/lang/Preconditions.java
@@ -1,19 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.utils.lang;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/utils/lang/Primitives.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/utils/lang/Primitives.java b/core/src/main/java/hivemall/utils/lang/Primitives.java
index d22b81d..8f018f0 100644
--- a/core/src/main/java/hivemall/utils/lang/Primitives.java
+++ b/core/src/main/java/hivemall/utils/lang/Primitives.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.utils.lang;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/utils/lang/PrivilegedAccessor.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/utils/lang/PrivilegedAccessor.java b/core/src/main/java/hivemall/utils/lang/PrivilegedAccessor.java
index f6b8441..234bb5b 100644
--- a/core/src/main/java/hivemall/utils/lang/PrivilegedAccessor.java
+++ b/core/src/main/java/hivemall/utils/lang/PrivilegedAccessor.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.utils.lang;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/utils/lang/RandomUtils.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/utils/lang/RandomUtils.java b/core/src/main/java/hivemall/utils/lang/RandomUtils.java
index f0751db..b7793f5 100644
--- a/core/src/main/java/hivemall/utils/lang/RandomUtils.java
+++ b/core/src/main/java/hivemall/utils/lang/RandomUtils.java
@@ -1,19 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.utils.lang;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/utils/lang/SizeOf.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/utils/lang/SizeOf.java b/core/src/main/java/hivemall/utils/lang/SizeOf.java
index 68f362a..9e0ef4c 100644
--- a/core/src/main/java/hivemall/utils/lang/SizeOf.java
+++ b/core/src/main/java/hivemall/utils/lang/SizeOf.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.utils.lang;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/utils/lang/StringUtils.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/utils/lang/StringUtils.java b/core/src/main/java/hivemall/utils/lang/StringUtils.java
index 16d92cb..48e137f 100644
--- a/core/src/main/java/hivemall/utils/lang/StringUtils.java
+++ b/core/src/main/java/hivemall/utils/lang/StringUtils.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.utils.lang;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/utils/lang/UnsafeUtils.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/utils/lang/UnsafeUtils.java b/core/src/main/java/hivemall/utils/lang/UnsafeUtils.java
index b04cb9f..ade437d 100644
--- a/core/src/main/java/hivemall/utils/lang/UnsafeUtils.java
+++ b/core/src/main/java/hivemall/utils/lang/UnsafeUtils.java
@@ -1,19 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.utils.lang;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/utils/lang/mutable/MutableDouble.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/utils/lang/mutable/MutableDouble.java b/core/src/main/java/hivemall/utils/lang/mutable/MutableDouble.java
index 95ffb46..4770882 100644
--- a/core/src/main/java/hivemall/utils/lang/mutable/MutableDouble.java
+++ b/core/src/main/java/hivemall/utils/lang/mutable/MutableDouble.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.utils.lang.mutable;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/utils/lang/mutable/MutableFloat.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/utils/lang/mutable/MutableFloat.java b/core/src/main/java/hivemall/utils/lang/mutable/MutableFloat.java
index 7ed8145..8ec3e99 100644
--- a/core/src/main/java/hivemall/utils/lang/mutable/MutableFloat.java
+++ b/core/src/main/java/hivemall/utils/lang/mutable/MutableFloat.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.utils.lang.mutable;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/utils/lang/mutable/MutableInt.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/utils/lang/mutable/MutableInt.java b/core/src/main/java/hivemall/utils/lang/mutable/MutableInt.java
index 1513953..904fa5e 100644
--- a/core/src/main/java/hivemall/utils/lang/mutable/MutableInt.java
+++ b/core/src/main/java/hivemall/utils/lang/mutable/MutableInt.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.utils.lang.mutable;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/utils/lang/mutable/MutableLong.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/utils/lang/mutable/MutableLong.java b/core/src/main/java/hivemall/utils/lang/mutable/MutableLong.java
index c68f026..06fb606 100644
--- a/core/src/main/java/hivemall/utils/lang/mutable/MutableLong.java
+++ b/core/src/main/java/hivemall/utils/lang/mutable/MutableLong.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.utils.lang.mutable;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/utils/lock/Lock.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/utils/lock/Lock.java b/core/src/main/java/hivemall/utils/lock/Lock.java
index 4a1dbb1..6b7e28b 100644
--- a/core/src/main/java/hivemall/utils/lock/Lock.java
+++ b/core/src/main/java/hivemall/utils/lock/Lock.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.utils.lock;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/utils/lock/TTASLock.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/utils/lock/TTASLock.java b/core/src/main/java/hivemall/utils/lock/TTASLock.java
index e68d97a..b36d1e1 100644
--- a/core/src/main/java/hivemall/utils/lock/TTASLock.java
+++ b/core/src/main/java/hivemall/utils/lock/TTASLock.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.utils.lock;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/utils/math/MathUtils.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/utils/math/MathUtils.java b/core/src/main/java/hivemall/utils/math/MathUtils.java
index aa30fdf..75ace9c 100644
--- a/core/src/main/java/hivemall/utils/math/MathUtils.java
+++ b/core/src/main/java/hivemall/utils/math/MathUtils.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.
  */
 //
 // Licensed to the Apache Software Foundation (ASF) under one or more

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/utils/math/MatrixUtils.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/utils/math/MatrixUtils.java b/core/src/main/java/hivemall/utils/math/MatrixUtils.java
index 840df41..9659526 100644
--- a/core/src/main/java/hivemall/utils/math/MatrixUtils.java
+++ b/core/src/main/java/hivemall/utils/math/MatrixUtils.java
@@ -1,19 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.utils.math;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/utils/math/Primes.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/utils/math/Primes.java b/core/src/main/java/hivemall/utils/math/Primes.java
index 7d48337..f65131a 100644
--- a/core/src/main/java/hivemall/utils/math/Primes.java
+++ b/core/src/main/java/hivemall/utils/math/Primes.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.utils.math;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/utils/math/StatsUtils.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/utils/math/StatsUtils.java b/core/src/main/java/hivemall/utils/math/StatsUtils.java
index 42a2c90..812f619 100644
--- a/core/src/main/java/hivemall/utils/math/StatsUtils.java
+++ b/core/src/main/java/hivemall/utils/math/StatsUtils.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.utils.math;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/utils/net/NetUtils.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/utils/net/NetUtils.java b/core/src/main/java/hivemall/utils/net/NetUtils.java
index 3d67c06..57fecf6 100644
--- a/core/src/main/java/hivemall/utils/net/NetUtils.java
+++ b/core/src/main/java/hivemall/utils/net/NetUtils.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.utils.net;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/org/apache/hadoop/hive/ql/exec/MapredContextAccessor.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/hadoop/hive/ql/exec/MapredContextAccessor.java b/core/src/main/java/org/apache/hadoop/hive/ql/exec/MapredContextAccessor.java
index 73e8702..5b760bd 100644
--- a/core/src/main/java/org/apache/hadoop/hive/ql/exec/MapredContextAccessor.java
+++ b/core/src/main/java/org/apache/hadoop/hive/ql/exec/MapredContextAccessor.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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 org.apache.hadoop.hive.ql.exec;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/test/java/hivemall/anomaly/ChangeFinder1DTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/hivemall/anomaly/ChangeFinder1DTest.java b/core/src/test/java/hivemall/anomaly/ChangeFinder1DTest.java
index e893fa4..d7bac75 100644
--- a/core/src/test/java/hivemall/anomaly/ChangeFinder1DTest.java
+++ b/core/src/test/java/hivemall/anomaly/ChangeFinder1DTest.java
@@ -1,19 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.anomaly;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/test/java/hivemall/anomaly/ChangeFinder2DTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/hivemall/anomaly/ChangeFinder2DTest.java b/core/src/test/java/hivemall/anomaly/ChangeFinder2DTest.java
index d927161..240906c 100644
--- a/core/src/test/java/hivemall/anomaly/ChangeFinder2DTest.java
+++ b/core/src/test/java/hivemall/anomaly/ChangeFinder2DTest.java
@@ -1,19 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.anomaly;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/test/java/hivemall/classifier/PassiveAggressiveUDTFTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/hivemall/classifier/PassiveAggressiveUDTFTest.java b/core/src/test/java/hivemall/classifier/PassiveAggressiveUDTFTest.java
index ce6ad70..2322c07 100644
--- a/core/src/test/java/hivemall/classifier/PassiveAggressiveUDTFTest.java
+++ b/core/src/test/java/hivemall/classifier/PassiveAggressiveUDTFTest.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.classifier;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/test/java/hivemall/classifier/PerceptronUDTFTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/hivemall/classifier/PerceptronUDTFTest.java b/core/src/test/java/hivemall/classifier/PerceptronUDTFTest.java
index 38a21a7..e04adb7 100644
--- a/core/src/test/java/hivemall/classifier/PerceptronUDTFTest.java
+++ b/core/src/test/java/hivemall/classifier/PerceptronUDTFTest.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.classifier;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/test/java/hivemall/common/OnlineVarianceTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/hivemall/common/OnlineVarianceTest.java b/core/src/test/java/hivemall/common/OnlineVarianceTest.java
index da8ed68..2308dea 100644
--- a/core/src/test/java/hivemall/common/OnlineVarianceTest.java
+++ b/core/src/test/java/hivemall/common/OnlineVarianceTest.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.common;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/test/java/hivemall/common/RandomizedAmplifierTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/hivemall/common/RandomizedAmplifierTest.java b/core/src/test/java/hivemall/common/RandomizedAmplifierTest.java
index 97235df..549434b 100644
--- a/core/src/test/java/hivemall/common/RandomizedAmplifierTest.java
+++ b/core/src/test/java/hivemall/common/RandomizedAmplifierTest.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.common;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/test/java/hivemall/evaluation/BinaryResponsesMeasuresTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/hivemall/evaluation/BinaryResponsesMeasuresTest.java b/core/src/test/java/hivemall/evaluation/BinaryResponsesMeasuresTest.java
index 1a96923..800eb6c 100644
--- a/core/src/test/java/hivemall/evaluation/BinaryResponsesMeasuresTest.java
+++ b/core/src/test/java/hivemall/evaluation/BinaryResponsesMeasuresTest.java
@@ -1,19 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.evaluation;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/test/java/hivemall/evaluation/GradedResponsesMeasuresTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/hivemall/evaluation/GradedResponsesMeasuresTest.java b/core/src/test/java/hivemall/evaluation/GradedResponsesMeasuresTest.java
index 669f0f6..765fa76 100644
--- a/core/src/test/java/hivemall/evaluation/GradedResponsesMeasuresTest.java
+++ b/core/src/test/java/hivemall/evaluation/GradedResponsesMeasuresTest.java
@@ -1,19 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.evaluation;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/test/java/hivemall/fm/ArrayModelTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/hivemall/fm/ArrayModelTest.java b/core/src/test/java/hivemall/fm/ArrayModelTest.java
index 9706dbd..a346c16 100644
--- a/core/src/test/java/hivemall/fm/ArrayModelTest.java
+++ b/core/src/test/java/hivemall/fm/ArrayModelTest.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.fm;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/test/java/hivemall/fm/FFMPredictionModelTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/hivemall/fm/FFMPredictionModelTest.java b/core/src/test/java/hivemall/fm/FFMPredictionModelTest.java
index 3ef206c..a65a69a 100644
--- a/core/src/test/java/hivemall/fm/FFMPredictionModelTest.java
+++ b/core/src/test/java/hivemall/fm/FFMPredictionModelTest.java
@@ -1,19 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.fm;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/test/java/hivemall/fm/FactorizationMachineUDTFTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/hivemall/fm/FactorizationMachineUDTFTest.java b/core/src/test/java/hivemall/fm/FactorizationMachineUDTFTest.java
index 81c1858..eacfa8d 100644
--- a/core/src/test/java/hivemall/fm/FactorizationMachineUDTFTest.java
+++ b/core/src/test/java/hivemall/fm/FactorizationMachineUDTFTest.java
@@ -1,3 +1,21 @@
+/*
+ * 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.fm;
 
 import java.io.BufferedReader;

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/test/java/hivemall/fm/FeatureTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/hivemall/fm/FeatureTest.java b/core/src/test/java/hivemall/fm/FeatureTest.java
index ec24384..25e5671 100644
--- a/core/src/test/java/hivemall/fm/FeatureTest.java
+++ b/core/src/test/java/hivemall/fm/FeatureTest.java
@@ -1,19 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.fm;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/test/java/hivemall/fm/FieldAwareFactorizationMachineUDTFTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/hivemall/fm/FieldAwareFactorizationMachineUDTFTest.java b/core/src/test/java/hivemall/fm/FieldAwareFactorizationMachineUDTFTest.java
index cc1fcf3..f4d4f80 100644
--- a/core/src/test/java/hivemall/fm/FieldAwareFactorizationMachineUDTFTest.java
+++ b/core/src/test/java/hivemall/fm/FieldAwareFactorizationMachineUDTFTest.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.fm;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/test/java/hivemall/fm/IntFeatureMapModelTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/hivemall/fm/IntFeatureMapModelTest.java b/core/src/test/java/hivemall/fm/IntFeatureMapModelTest.java
index 017c25b..890962f 100644
--- a/core/src/test/java/hivemall/fm/IntFeatureMapModelTest.java
+++ b/core/src/test/java/hivemall/fm/IntFeatureMapModelTest.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.fm;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/test/java/hivemall/fm/StringFeatureMapModelTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/hivemall/fm/StringFeatureMapModelTest.java b/core/src/test/java/hivemall/fm/StringFeatureMapModelTest.java
index 56fa137..40fb66e 100644
--- a/core/src/test/java/hivemall/fm/StringFeatureMapModelTest.java
+++ b/core/src/test/java/hivemall/fm/StringFeatureMapModelTest.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.fm;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/test/java/hivemall/ftvec/ExtractWeightUDFTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/hivemall/ftvec/ExtractWeightUDFTest.java b/core/src/test/java/hivemall/ftvec/ExtractWeightUDFTest.java
index a8685d9..5943e0b 100644
--- a/core/src/test/java/hivemall/ftvec/ExtractWeightUDFTest.java
+++ b/core/src/test/java/hivemall/ftvec/ExtractWeightUDFTest.java
@@ -1,19 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * 
- * Licensed 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
  *
- *         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.
+ * 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.ftvec;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/test/java/hivemall/ftvec/FeatureUDFTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/hivemall/ftvec/FeatureUDFTest.java b/core/src/test/java/hivemall/ftvec/FeatureUDFTest.java
index 8d536a5..0eb1289 100644
--- a/core/src/test/java/hivemall/ftvec/FeatureUDFTest.java
+++ b/core/src/test/java/hivemall/ftvec/FeatureUDFTest.java
@@ -1,19 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.ftvec;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/test/java/hivemall/ftvec/hashing/FeatureHashingUDFTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/hivemall/ftvec/hashing/FeatureHashingUDFTest.java b/core/src/test/java/hivemall/ftvec/hashing/FeatureHashingUDFTest.java
index 70e0b9c..061a971 100644
--- a/core/src/test/java/hivemall/ftvec/hashing/FeatureHashingUDFTest.java
+++ b/core/src/test/java/hivemall/ftvec/hashing/FeatureHashingUDFTest.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.ftvec.hashing;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/test/java/hivemall/ftvec/hashing/MurmurHash3UDFTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/hivemall/ftvec/hashing/MurmurHash3UDFTest.java b/core/src/test/java/hivemall/ftvec/hashing/MurmurHash3UDFTest.java
index bfbfe2b..ec22106 100644
--- a/core/src/test/java/hivemall/ftvec/hashing/MurmurHash3UDFTest.java
+++ b/core/src/test/java/hivemall/ftvec/hashing/MurmurHash3UDFTest.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.ftvec.hashing;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/test/java/hivemall/ftvec/hashing/Sha1UDFTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/hivemall/ftvec/hashing/Sha1UDFTest.java b/core/src/test/java/hivemall/ftvec/hashing/Sha1UDFTest.java
index 36c4dae..b2057f1 100644
--- a/core/src/test/java/hivemall/ftvec/hashing/Sha1UDFTest.java
+++ b/core/src/test/java/hivemall/ftvec/hashing/Sha1UDFTest.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.ftvec.hashing;
 



[31/50] [abbrv] incubator-hivemall git commit: Updated license header of GenericUDAFMaxRow to follow https://github.com/scribd/hive-udaf-maxrow/blob/master/src/com/scribd/hive/udaf/GenericUDAFMaxRow.java

Posted by my...@apache.org.
Updated license header of GenericUDAFMaxRow to follow
https://github.com/scribd/hive-udaf-maxrow/blob/master/src/com/scribd/hive/udaf/GenericUDAFMaxRow.java

Project: http://git-wip-us.apache.org/repos/asf/incubator-hivemall/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-hivemall/commit/0cf2a645
Tree: http://git-wip-us.apache.org/repos/asf/incubator-hivemall/tree/0cf2a645
Diff: http://git-wip-us.apache.org/repos/asf/incubator-hivemall/diff/0cf2a645

Branch: refs/heads/master
Commit: 0cf2a64541f75f7b874eeb12424e59cf82627a66
Parents: 63aa3b9
Author: myui <yu...@gmail.com>
Authored: Sat Nov 12 14:46:31 2016 +0900
Committer: myui <yu...@gmail.com>
Committed: Sat Nov 12 14:47:46 2016 +0900

----------------------------------------------------------------------
 NOTICE                                          | 36 ++++++++++++++++----
 .../main/java/hivemall/ensemble/MaxRowUDAF.java | 15 --------
 2 files changed, 30 insertions(+), 21 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/0cf2a645/NOTICE
----------------------------------------------------------------------
diff --git a/NOTICE b/NOTICE
index 68e506f..c65fa03 100644
--- a/NOTICE
+++ b/NOTICE
@@ -1,9 +1,33 @@
-Hivemall: Hive scalable machine learning library
+Apache Hivemall
+Copyright 2016 The Apache Software Foundation
 
-Copyright (C) 2013 National Institute of Advanced Industrial Science and Technology (AIST)
+This product includes software developed at
+The Apache Software Foundation (http://www.apache.org/).
 
-This product includes/uses software developed by The Apache Software Foundation (http://www.apache.org/).
-The Apache Hive project logo are trademarks of The Apache Software Foundation.
+Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+Copyright (C) 2015-2016 Makoto Yui
 
-Put the above copyrights and LICENSE file for the services/softwares/distributions 
-that use Hivemall.
+------------------------------------------------------------------------------------------------------
+Copyright notifications which have been relocated from source files
+
+o hivemall/core/src/main/java/hivemall/smile/classification/DecisionTree.java
+  hivemall/core/src/main/java/hivemall/smile/regression/RegressionTree.java
+
+    Copyright (c) 2010 Haifeng Li
+
+    https://github.com/haifengl/smile
+    Licensed under the Apache License, Version 2.0
+
+o hivemall/core/src/main/java/hivemall/utils/codec/Base91.java
+
+    Copyright (c) 2000-2006 Joachim Henke
+
+    https://github.com/bwaldvogel/base91    
+    Licensed under the BSD 3-Clause License
+
+o hivemall/core/src/main/java/hivemall/utils/collections/OpenHashMap.java
+
+    Copyright (C) 2010 catchpole.net
+
+    https://github.com/slipperyseal/atomicobjects/
+    Licensed under the Apache License, Version 2.0

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/0cf2a645/core/src/main/java/hivemall/ensemble/MaxRowUDAF.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/ensemble/MaxRowUDAF.java b/core/src/main/java/hivemall/ensemble/MaxRowUDAF.java
index f30d80a..29d1563 100644
--- a/core/src/main/java/hivemall/ensemble/MaxRowUDAF.java
+++ b/core/src/main/java/hivemall/ensemble/MaxRowUDAF.java
@@ -16,21 +16,6 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-//
-//   Copyright (C) 2013 Michael Birk ( https://github.com/scribd/hive-udaf-maxrow )
-//
-//   Licensed 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.ensemble;
 
 import java.util.ArrayList;


[21/50] [abbrv] incubator-hivemall git commit: Updated license headers

Posted by my...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/common/LossFunctions.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/common/LossFunctions.java b/core/src/main/java/hivemall/common/LossFunctions.java
index 6b403fd..e1a0f31 100644
--- a/core/src/main/java/hivemall/common/LossFunctions.java
+++ b/core/src/main/java/hivemall/common/LossFunctions.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.common;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/common/OnlineVariance.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/common/OnlineVariance.java b/core/src/main/java/hivemall/common/OnlineVariance.java
index 449ea56..6e1d990 100644
--- a/core/src/main/java/hivemall/common/OnlineVariance.java
+++ b/core/src/main/java/hivemall/common/OnlineVariance.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.common;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/common/RandomizedAmplifier.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/common/RandomizedAmplifier.java b/core/src/main/java/hivemall/common/RandomizedAmplifier.java
index 993cc44..7f37ec5 100644
--- a/core/src/main/java/hivemall/common/RandomizedAmplifier.java
+++ b/core/src/main/java/hivemall/common/RandomizedAmplifier.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.common;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/common/ReservoirSampler.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/common/ReservoirSampler.java b/core/src/main/java/hivemall/common/ReservoirSampler.java
index 752caa7..8846ac1 100644
--- a/core/src/main/java/hivemall/common/ReservoirSampler.java
+++ b/core/src/main/java/hivemall/common/ReservoirSampler.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.common;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/dataset/LogisticRegressionDataGeneratorUDTF.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/dataset/LogisticRegressionDataGeneratorUDTF.java b/core/src/main/java/hivemall/dataset/LogisticRegressionDataGeneratorUDTF.java
index a7cb534..3da1fd0 100644
--- a/core/src/main/java/hivemall/dataset/LogisticRegressionDataGeneratorUDTF.java
+++ b/core/src/main/java/hivemall/dataset/LogisticRegressionDataGeneratorUDTF.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.dataset;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/ensemble/ArgminKLDistanceUDAF.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/ensemble/ArgminKLDistanceUDAF.java b/core/src/main/java/hivemall/ensemble/ArgminKLDistanceUDAF.java
index 0a94299..d6cd7d4 100644
--- a/core/src/main/java/hivemall/ensemble/ArgminKLDistanceUDAF.java
+++ b/core/src/main/java/hivemall/ensemble/ArgminKLDistanceUDAF.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.ensemble;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/ensemble/MaxRowUDAF.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/ensemble/MaxRowUDAF.java b/core/src/main/java/hivemall/ensemble/MaxRowUDAF.java
index cf5bab0..f30d80a 100644
--- a/core/src/main/java/hivemall/ensemble/MaxRowUDAF.java
+++ b/core/src/main/java/hivemall/ensemble/MaxRowUDAF.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.
  */
 //
 //   Copyright (C) 2013 Michael Birk ( https://github.com/scribd/hive-udaf-maxrow )

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/ensemble/MaxValueLabelUDAF.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/ensemble/MaxValueLabelUDAF.java b/core/src/main/java/hivemall/ensemble/MaxValueLabelUDAF.java
index 935ad09..6b5266a 100644
--- a/core/src/main/java/hivemall/ensemble/MaxValueLabelUDAF.java
+++ b/core/src/main/java/hivemall/ensemble/MaxValueLabelUDAF.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.ensemble;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/ensemble/bagging/VotedAvgUDAF.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/ensemble/bagging/VotedAvgUDAF.java b/core/src/main/java/hivemall/ensemble/bagging/VotedAvgUDAF.java
index 7bef34d..8b40142 100644
--- a/core/src/main/java/hivemall/ensemble/bagging/VotedAvgUDAF.java
+++ b/core/src/main/java/hivemall/ensemble/bagging/VotedAvgUDAF.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.ensemble.bagging;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/ensemble/bagging/WeightVotedAvgUDAF.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/ensemble/bagging/WeightVotedAvgUDAF.java b/core/src/main/java/hivemall/ensemble/bagging/WeightVotedAvgUDAF.java
index 41ab485..a7d63be 100644
--- a/core/src/main/java/hivemall/ensemble/bagging/WeightVotedAvgUDAF.java
+++ b/core/src/main/java/hivemall/ensemble/bagging/WeightVotedAvgUDAF.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.ensemble.bagging;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/evaluation/AUCUDAF.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/evaluation/AUCUDAF.java b/core/src/main/java/hivemall/evaluation/AUCUDAF.java
index d2369d3..26f1fbc 100644
--- a/core/src/main/java/hivemall/evaluation/AUCUDAF.java
+++ b/core/src/main/java/hivemall/evaluation/AUCUDAF.java
@@ -1,19 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.evaluation;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/evaluation/BinaryResponsesMeasures.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/evaluation/BinaryResponsesMeasures.java b/core/src/main/java/hivemall/evaluation/BinaryResponsesMeasures.java
index 14f502b..fc25882 100644
--- a/core/src/main/java/hivemall/evaluation/BinaryResponsesMeasures.java
+++ b/core/src/main/java/hivemall/evaluation/BinaryResponsesMeasures.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.evaluation;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/evaluation/FMeasureUDAF.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/evaluation/FMeasureUDAF.java b/core/src/main/java/hivemall/evaluation/FMeasureUDAF.java
index 65c5b3a..5d41cb8 100644
--- a/core/src/main/java/hivemall/evaluation/FMeasureUDAF.java
+++ b/core/src/main/java/hivemall/evaluation/FMeasureUDAF.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.evaluation;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/evaluation/GradedResponsesMeasures.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/evaluation/GradedResponsesMeasures.java b/core/src/main/java/hivemall/evaluation/GradedResponsesMeasures.java
index 9d0fac2..688ba53 100644
--- a/core/src/main/java/hivemall/evaluation/GradedResponsesMeasures.java
+++ b/core/src/main/java/hivemall/evaluation/GradedResponsesMeasures.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.evaluation;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/evaluation/LogarithmicLossUDAF.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/evaluation/LogarithmicLossUDAF.java b/core/src/main/java/hivemall/evaluation/LogarithmicLossUDAF.java
index c0deed7..9cea6fb 100644
--- a/core/src/main/java/hivemall/evaluation/LogarithmicLossUDAF.java
+++ b/core/src/main/java/hivemall/evaluation/LogarithmicLossUDAF.java
@@ -1,19 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.evaluation;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/evaluation/MAPUDAF.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/evaluation/MAPUDAF.java b/core/src/main/java/hivemall/evaluation/MAPUDAF.java
index 44050c4..1b2f055 100644
--- a/core/src/main/java/hivemall/evaluation/MAPUDAF.java
+++ b/core/src/main/java/hivemall/evaluation/MAPUDAF.java
@@ -1,19 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.evaluation;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/evaluation/MRRUDAF.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/evaluation/MRRUDAF.java b/core/src/main/java/hivemall/evaluation/MRRUDAF.java
index 066ce5e..8778c18 100644
--- a/core/src/main/java/hivemall/evaluation/MRRUDAF.java
+++ b/core/src/main/java/hivemall/evaluation/MRRUDAF.java
@@ -1,19 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.evaluation;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/evaluation/MeanAbsoluteErrorUDAF.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/evaluation/MeanAbsoluteErrorUDAF.java b/core/src/main/java/hivemall/evaluation/MeanAbsoluteErrorUDAF.java
index eb7cccc..3a91984 100644
--- a/core/src/main/java/hivemall/evaluation/MeanAbsoluteErrorUDAF.java
+++ b/core/src/main/java/hivemall/evaluation/MeanAbsoluteErrorUDAF.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.evaluation;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/evaluation/MeanSquaredErrorUDAF.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/evaluation/MeanSquaredErrorUDAF.java b/core/src/main/java/hivemall/evaluation/MeanSquaredErrorUDAF.java
index 912c082..0d741f6 100644
--- a/core/src/main/java/hivemall/evaluation/MeanSquaredErrorUDAF.java
+++ b/core/src/main/java/hivemall/evaluation/MeanSquaredErrorUDAF.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.evaluation;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/evaluation/NDCGUDAF.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/evaluation/NDCGUDAF.java b/core/src/main/java/hivemall/evaluation/NDCGUDAF.java
index 3dbd141..e94e26c 100644
--- a/core/src/main/java/hivemall/evaluation/NDCGUDAF.java
+++ b/core/src/main/java/hivemall/evaluation/NDCGUDAF.java
@@ -1,19 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.evaluation;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/evaluation/PrecisionUDAF.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/evaluation/PrecisionUDAF.java b/core/src/main/java/hivemall/evaluation/PrecisionUDAF.java
index 3218708..335a155 100644
--- a/core/src/main/java/hivemall/evaluation/PrecisionUDAF.java
+++ b/core/src/main/java/hivemall/evaluation/PrecisionUDAF.java
@@ -1,19 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.evaluation;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/evaluation/R2UDAF.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/evaluation/R2UDAF.java b/core/src/main/java/hivemall/evaluation/R2UDAF.java
index f486aec..4c8231d 100755
--- a/core/src/main/java/hivemall/evaluation/R2UDAF.java
+++ b/core/src/main/java/hivemall/evaluation/R2UDAF.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.evaluation;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/evaluation/RecallUDAF.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/evaluation/RecallUDAF.java b/core/src/main/java/hivemall/evaluation/RecallUDAF.java
index 786c121..e268fd7 100644
--- a/core/src/main/java/hivemall/evaluation/RecallUDAF.java
+++ b/core/src/main/java/hivemall/evaluation/RecallUDAF.java
@@ -1,19 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.evaluation;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/evaluation/RootMeanSquaredErrorUDAF.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/evaluation/RootMeanSquaredErrorUDAF.java b/core/src/main/java/hivemall/evaluation/RootMeanSquaredErrorUDAF.java
index 11b95ff..5ce4d48 100644
--- a/core/src/main/java/hivemall/evaluation/RootMeanSquaredErrorUDAF.java
+++ b/core/src/main/java/hivemall/evaluation/RootMeanSquaredErrorUDAF.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.evaluation;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/fm/Entry.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/fm/Entry.java b/core/src/main/java/hivemall/fm/Entry.java
index f65ce62..1882f85 100644
--- a/core/src/main/java/hivemall/fm/Entry.java
+++ b/core/src/main/java/hivemall/fm/Entry.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.fm;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/fm/FFMPredictUDF.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/fm/FFMPredictUDF.java b/core/src/main/java/hivemall/fm/FFMPredictUDF.java
index ddbf14b..e32c61a 100644
--- a/core/src/main/java/hivemall/fm/FFMPredictUDF.java
+++ b/core/src/main/java/hivemall/fm/FFMPredictUDF.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.fm;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/fm/FFMPredictionModel.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/fm/FFMPredictionModel.java b/core/src/main/java/hivemall/fm/FFMPredictionModel.java
index b4c74ae..6969d05 100644
--- a/core/src/main/java/hivemall/fm/FFMPredictionModel.java
+++ b/core/src/main/java/hivemall/fm/FFMPredictionModel.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.fm;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/fm/FFMStringFeatureMapModel.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/fm/FFMStringFeatureMapModel.java b/core/src/main/java/hivemall/fm/FFMStringFeatureMapModel.java
index b93c857..4009326 100644
--- a/core/src/main/java/hivemall/fm/FFMStringFeatureMapModel.java
+++ b/core/src/main/java/hivemall/fm/FFMStringFeatureMapModel.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.fm;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/fm/FMArrayModel.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/fm/FMArrayModel.java b/core/src/main/java/hivemall/fm/FMArrayModel.java
index 30ee95e..fbae404 100644
--- a/core/src/main/java/hivemall/fm/FMArrayModel.java
+++ b/core/src/main/java/hivemall/fm/FMArrayModel.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.fm;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/fm/FMHyperParameters.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/fm/FMHyperParameters.java b/core/src/main/java/hivemall/fm/FMHyperParameters.java
index db69db3..a1e4d25 100644
--- a/core/src/main/java/hivemall/fm/FMHyperParameters.java
+++ b/core/src/main/java/hivemall/fm/FMHyperParameters.java
@@ -1,19 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.fm;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/fm/FMIntFeatureMapModel.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/fm/FMIntFeatureMapModel.java b/core/src/main/java/hivemall/fm/FMIntFeatureMapModel.java
index e23b33f..d2a5ed6 100644
--- a/core/src/main/java/hivemall/fm/FMIntFeatureMapModel.java
+++ b/core/src/main/java/hivemall/fm/FMIntFeatureMapModel.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.fm;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/fm/FMPredictGenericUDAF.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/fm/FMPredictGenericUDAF.java b/core/src/main/java/hivemall/fm/FMPredictGenericUDAF.java
index c36f6b6..667befb 100644
--- a/core/src/main/java/hivemall/fm/FMPredictGenericUDAF.java
+++ b/core/src/main/java/hivemall/fm/FMPredictGenericUDAF.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.fm;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/fm/FMStringFeatureMapModel.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/fm/FMStringFeatureMapModel.java b/core/src/main/java/hivemall/fm/FMStringFeatureMapModel.java
index 3a8ae80..10ffaae 100644
--- a/core/src/main/java/hivemall/fm/FMStringFeatureMapModel.java
+++ b/core/src/main/java/hivemall/fm/FMStringFeatureMapModel.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.fm;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/fm/FactorizationMachineModel.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/fm/FactorizationMachineModel.java b/core/src/main/java/hivemall/fm/FactorizationMachineModel.java
index 396328a..a7fbb4e 100644
--- a/core/src/main/java/hivemall/fm/FactorizationMachineModel.java
+++ b/core/src/main/java/hivemall/fm/FactorizationMachineModel.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.fm;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/fm/FactorizationMachineUDTF.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/fm/FactorizationMachineUDTF.java b/core/src/main/java/hivemall/fm/FactorizationMachineUDTF.java
index 2388689..d263cbc 100644
--- a/core/src/main/java/hivemall/fm/FactorizationMachineUDTF.java
+++ b/core/src/main/java/hivemall/fm/FactorizationMachineUDTF.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.fm;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/fm/Feature.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/fm/Feature.java b/core/src/main/java/hivemall/fm/Feature.java
index 251b80d..f2d977e 100644
--- a/core/src/main/java/hivemall/fm/Feature.java
+++ b/core/src/main/java/hivemall/fm/Feature.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.fm;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/fm/FieldAwareFactorizationMachineModel.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/fm/FieldAwareFactorizationMachineModel.java b/core/src/main/java/hivemall/fm/FieldAwareFactorizationMachineModel.java
index 7e3cc50..e63797c 100644
--- a/core/src/main/java/hivemall/fm/FieldAwareFactorizationMachineModel.java
+++ b/core/src/main/java/hivemall/fm/FieldAwareFactorizationMachineModel.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.fm;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/fm/FieldAwareFactorizationMachineUDTF.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/fm/FieldAwareFactorizationMachineUDTF.java b/core/src/main/java/hivemall/fm/FieldAwareFactorizationMachineUDTF.java
index d99bee9..62fbfa1 100644
--- a/core/src/main/java/hivemall/fm/FieldAwareFactorizationMachineUDTF.java
+++ b/core/src/main/java/hivemall/fm/FieldAwareFactorizationMachineUDTF.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.fm;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/fm/IntFeature.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/fm/IntFeature.java b/core/src/main/java/hivemall/fm/IntFeature.java
index 2a9f8bd..2052f7e 100644
--- a/core/src/main/java/hivemall/fm/IntFeature.java
+++ b/core/src/main/java/hivemall/fm/IntFeature.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.fm;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/fm/StringFeature.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/fm/StringFeature.java b/core/src/main/java/hivemall/fm/StringFeature.java
index e74cf6c..587ae45 100644
--- a/core/src/main/java/hivemall/fm/StringFeature.java
+++ b/core/src/main/java/hivemall/fm/StringFeature.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.fm;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/ftvec/AddBiasUDF.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/ftvec/AddBiasUDF.java b/core/src/main/java/hivemall/ftvec/AddBiasUDF.java
index 222d946..5870a4f 100644
--- a/core/src/main/java/hivemall/ftvec/AddBiasUDF.java
+++ b/core/src/main/java/hivemall/ftvec/AddBiasUDF.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.ftvec;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/ftvec/AddFeatureIndexUDF.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/ftvec/AddFeatureIndexUDF.java b/core/src/main/java/hivemall/ftvec/AddFeatureIndexUDF.java
index 5c8ff2f..105dd2a 100644
--- a/core/src/main/java/hivemall/ftvec/AddFeatureIndexUDF.java
+++ b/core/src/main/java/hivemall/ftvec/AddFeatureIndexUDF.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.ftvec;
 


[17/50] [abbrv] incubator-hivemall git commit: Updated license headers

Posted by my...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/tools/bits/BitsORUDF.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/tools/bits/BitsORUDF.java b/core/src/main/java/hivemall/tools/bits/BitsORUDF.java
index 909628a..98a659c 100644
--- a/core/src/main/java/hivemall/tools/bits/BitsORUDF.java
+++ b/core/src/main/java/hivemall/tools/bits/BitsORUDF.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.tools.bits;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/tools/bits/ToBitsUDF.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/tools/bits/ToBitsUDF.java b/core/src/main/java/hivemall/tools/bits/ToBitsUDF.java
index 0ebda7f..ba9bcd9 100644
--- a/core/src/main/java/hivemall/tools/bits/ToBitsUDF.java
+++ b/core/src/main/java/hivemall/tools/bits/ToBitsUDF.java
@@ -1,3 +1,21 @@
+/*
+ * 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.tools.bits;
 
 import hivemall.utils.hadoop.HiveUtils;

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/tools/bits/UnBitsUDF.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/tools/bits/UnBitsUDF.java b/core/src/main/java/hivemall/tools/bits/UnBitsUDF.java
index 1525972..705d19d 100644
--- a/core/src/main/java/hivemall/tools/bits/UnBitsUDF.java
+++ b/core/src/main/java/hivemall/tools/bits/UnBitsUDF.java
@@ -1,3 +1,21 @@
+/*
+ * 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.tools.bits;
 
 import hivemall.utils.hadoop.HiveUtils;

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/tools/compress/DeflateUDF.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/tools/compress/DeflateUDF.java b/core/src/main/java/hivemall/tools/compress/DeflateUDF.java
index e34c925..26c4c0e 100644
--- a/core/src/main/java/hivemall/tools/compress/DeflateUDF.java
+++ b/core/src/main/java/hivemall/tools/compress/DeflateUDF.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.tools.compress;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/tools/compress/InflateUDF.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/tools/compress/InflateUDF.java b/core/src/main/java/hivemall/tools/compress/InflateUDF.java
index f4afa62..0021208 100644
--- a/core/src/main/java/hivemall/tools/compress/InflateUDF.java
+++ b/core/src/main/java/hivemall/tools/compress/InflateUDF.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.tools.compress;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/tools/map/MapGetSumUDF.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/tools/map/MapGetSumUDF.java b/core/src/main/java/hivemall/tools/map/MapGetSumUDF.java
index e53f919..a511963 100644
--- a/core/src/main/java/hivemall/tools/map/MapGetSumUDF.java
+++ b/core/src/main/java/hivemall/tools/map/MapGetSumUDF.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.tools.map;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/tools/map/MapTailNUDF.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/tools/map/MapTailNUDF.java b/core/src/main/java/hivemall/tools/map/MapTailNUDF.java
index 26c10bc..f2da8b8 100644
--- a/core/src/main/java/hivemall/tools/map/MapTailNUDF.java
+++ b/core/src/main/java/hivemall/tools/map/MapTailNUDF.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.tools.map;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/tools/map/UDAFToMap.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/tools/map/UDAFToMap.java b/core/src/main/java/hivemall/tools/map/UDAFToMap.java
index 5b1c89a..e4d42d9 100644
--- a/core/src/main/java/hivemall/tools/map/UDAFToMap.java
+++ b/core/src/main/java/hivemall/tools/map/UDAFToMap.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.tools.map;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/tools/map/UDAFToOrderedMap.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/tools/map/UDAFToOrderedMap.java b/core/src/main/java/hivemall/tools/map/UDAFToOrderedMap.java
index c06cbd8..1acc67e 100644
--- a/core/src/main/java/hivemall/tools/map/UDAFToOrderedMap.java
+++ b/core/src/main/java/hivemall/tools/map/UDAFToOrderedMap.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.tools.map;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/tools/mapred/DistributedCacheLookupUDF.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/tools/mapred/DistributedCacheLookupUDF.java b/core/src/main/java/hivemall/tools/mapred/DistributedCacheLookupUDF.java
index 1904309..1f6c324 100644
--- a/core/src/main/java/hivemall/tools/mapred/DistributedCacheLookupUDF.java
+++ b/core/src/main/java/hivemall/tools/mapred/DistributedCacheLookupUDF.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.tools.mapred;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/tools/mapred/JobConfGetsUDF.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/tools/mapred/JobConfGetsUDF.java b/core/src/main/java/hivemall/tools/mapred/JobConfGetsUDF.java
index 06481b3..7c978d0 100644
--- a/core/src/main/java/hivemall/tools/mapred/JobConfGetsUDF.java
+++ b/core/src/main/java/hivemall/tools/mapred/JobConfGetsUDF.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.tools.mapred;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/tools/mapred/JobIdUDF.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/tools/mapred/JobIdUDF.java b/core/src/main/java/hivemall/tools/mapred/JobIdUDF.java
index a7c5a72..d4c962e 100644
--- a/core/src/main/java/hivemall/tools/mapred/JobIdUDF.java
+++ b/core/src/main/java/hivemall/tools/mapred/JobIdUDF.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.tools.mapred;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/tools/mapred/RowIdUDF.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/tools/mapred/RowIdUDF.java b/core/src/main/java/hivemall/tools/mapred/RowIdUDF.java
index 307bff0..558bb68 100644
--- a/core/src/main/java/hivemall/tools/mapred/RowIdUDF.java
+++ b/core/src/main/java/hivemall/tools/mapred/RowIdUDF.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.tools.mapred;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/tools/mapred/TaskIdUDF.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/tools/mapred/TaskIdUDF.java b/core/src/main/java/hivemall/tools/mapred/TaskIdUDF.java
index 8f31f02..157a5a4 100644
--- a/core/src/main/java/hivemall/tools/mapred/TaskIdUDF.java
+++ b/core/src/main/java/hivemall/tools/mapred/TaskIdUDF.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.tools.mapred;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/tools/math/SigmoidGenericUDF.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/tools/math/SigmoidGenericUDF.java b/core/src/main/java/hivemall/tools/math/SigmoidGenericUDF.java
index 485a918..881d94d 100644
--- a/core/src/main/java/hivemall/tools/math/SigmoidGenericUDF.java
+++ b/core/src/main/java/hivemall/tools/math/SigmoidGenericUDF.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.tools.math;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/tools/text/Base91UDF.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/tools/text/Base91UDF.java b/core/src/main/java/hivemall/tools/text/Base91UDF.java
index c55e1d1..44d6b22 100644
--- a/core/src/main/java/hivemall/tools/text/Base91UDF.java
+++ b/core/src/main/java/hivemall/tools/text/Base91UDF.java
@@ -1,19 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.tools.text;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/tools/text/NormalizeUnicodeUDF.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/tools/text/NormalizeUnicodeUDF.java b/core/src/main/java/hivemall/tools/text/NormalizeUnicodeUDF.java
index 8b3f878..d34225d 100644
--- a/core/src/main/java/hivemall/tools/text/NormalizeUnicodeUDF.java
+++ b/core/src/main/java/hivemall/tools/text/NormalizeUnicodeUDF.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.tools.text;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/tools/text/SplitWordsUDF.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/tools/text/SplitWordsUDF.java b/core/src/main/java/hivemall/tools/text/SplitWordsUDF.java
index 2a5b01f..3f318f2 100644
--- a/core/src/main/java/hivemall/tools/text/SplitWordsUDF.java
+++ b/core/src/main/java/hivemall/tools/text/SplitWordsUDF.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.tools.text;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/tools/text/StopwordUDF.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/tools/text/StopwordUDF.java b/core/src/main/java/hivemall/tools/text/StopwordUDF.java
index 482b91f..9d68de1 100644
--- a/core/src/main/java/hivemall/tools/text/StopwordUDF.java
+++ b/core/src/main/java/hivemall/tools/text/StopwordUDF.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.tools.text;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/tools/text/TokenizeUDF.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/tools/text/TokenizeUDF.java b/core/src/main/java/hivemall/tools/text/TokenizeUDF.java
index 7b2cda4..a5c8777 100644
--- a/core/src/main/java/hivemall/tools/text/TokenizeUDF.java
+++ b/core/src/main/java/hivemall/tools/text/TokenizeUDF.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.tools.text;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/tools/text/Unbase91UDF.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/tools/text/Unbase91UDF.java b/core/src/main/java/hivemall/tools/text/Unbase91UDF.java
index fd421f4..32b1098 100644
--- a/core/src/main/java/hivemall/tools/text/Unbase91UDF.java
+++ b/core/src/main/java/hivemall/tools/text/Unbase91UDF.java
@@ -1,19 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.tools.text;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/utils/HivemallUtils.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/utils/HivemallUtils.java b/core/src/main/java/hivemall/utils/HivemallUtils.java
index fdbaf0b..d8d8979 100644
--- a/core/src/main/java/hivemall/utils/HivemallUtils.java
+++ b/core/src/main/java/hivemall/utils/HivemallUtils.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.utils;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/utils/buffer/DynamicByteArray.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/utils/buffer/DynamicByteArray.java b/core/src/main/java/hivemall/utils/buffer/DynamicByteArray.java
index c1e710e..87a8fbb 100644
--- a/core/src/main/java/hivemall/utils/buffer/DynamicByteArray.java
+++ b/core/src/main/java/hivemall/utils/buffer/DynamicByteArray.java
@@ -1,19 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
- *
- * Copyright (C) 2015 Makoto YUI
- *
- * Licensed 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
+ * 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
+ *   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.
+ * 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.
  */
 /*
  * Licensed to the Apache Software Foundation (ASF) under one

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/utils/buffer/HeapBuffer.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/utils/buffer/HeapBuffer.java b/core/src/main/java/hivemall/utils/buffer/HeapBuffer.java
index e7075f8..e0a3c9e 100644
--- a/core/src/main/java/hivemall/utils/buffer/HeapBuffer.java
+++ b/core/src/main/java/hivemall/utils/buffer/HeapBuffer.java
@@ -1,19 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.utils.buffer;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/utils/codec/Base91.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/utils/codec/Base91.java b/core/src/main/java/hivemall/utils/codec/Base91.java
index 4e25fc0..27bdf62 100644
--- a/core/src/main/java/hivemall/utils/codec/Base91.java
+++ b/core/src/main/java/hivemall/utils/codec/Base91.java
@@ -1,19 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.
  */
 /*
  * basE91 encoding/decoding routines

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/utils/codec/CompressionCodec.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/utils/codec/CompressionCodec.java b/core/src/main/java/hivemall/utils/codec/CompressionCodec.java
index 1355205..9b6d33d 100644
--- a/core/src/main/java/hivemall/utils/codec/CompressionCodec.java
+++ b/core/src/main/java/hivemall/utils/codec/CompressionCodec.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.utils.codec;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/utils/codec/DeflateCodec.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/utils/codec/DeflateCodec.java b/core/src/main/java/hivemall/utils/codec/DeflateCodec.java
index d4a63a9..7697f30 100644
--- a/core/src/main/java/hivemall/utils/codec/DeflateCodec.java
+++ b/core/src/main/java/hivemall/utils/codec/DeflateCodec.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.utils.codec;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/utils/codec/VariableByteCodec.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/utils/codec/VariableByteCodec.java b/core/src/main/java/hivemall/utils/codec/VariableByteCodec.java
index 01a703a..376332a 100644
--- a/core/src/main/java/hivemall/utils/codec/VariableByteCodec.java
+++ b/core/src/main/java/hivemall/utils/codec/VariableByteCodec.java
@@ -1,19 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.utils.codec;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/utils/codec/ZigZagLEB128Codec.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/utils/codec/ZigZagLEB128Codec.java b/core/src/main/java/hivemall/utils/codec/ZigZagLEB128Codec.java
index 9518a69..21ea118 100644
--- a/core/src/main/java/hivemall/utils/codec/ZigZagLEB128Codec.java
+++ b/core/src/main/java/hivemall/utils/codec/ZigZagLEB128Codec.java
@@ -1,19 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.utils.codec;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/utils/collections/BoundedPriorityQueue.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/utils/collections/BoundedPriorityQueue.java b/core/src/main/java/hivemall/utils/collections/BoundedPriorityQueue.java
index 232ebab..6844e81 100644
--- a/core/src/main/java/hivemall/utils/collections/BoundedPriorityQueue.java
+++ b/core/src/main/java/hivemall/utils/collections/BoundedPriorityQueue.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.utils.collections;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/utils/collections/DoubleArray3D.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/utils/collections/DoubleArray3D.java b/core/src/main/java/hivemall/utils/collections/DoubleArray3D.java
index d61a163..5716212 100644
--- a/core/src/main/java/hivemall/utils/collections/DoubleArray3D.java
+++ b/core/src/main/java/hivemall/utils/collections/DoubleArray3D.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.utils.collections;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/utils/collections/DoubleArrayList.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/utils/collections/DoubleArrayList.java b/core/src/main/java/hivemall/utils/collections/DoubleArrayList.java
index d35e6f6..1a37845 100644
--- a/core/src/main/java/hivemall/utils/collections/DoubleArrayList.java
+++ b/core/src/main/java/hivemall/utils/collections/DoubleArrayList.java
@@ -1,19 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.utils.collections;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/utils/collections/DoubleRingBuffer.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/utils/collections/DoubleRingBuffer.java b/core/src/main/java/hivemall/utils/collections/DoubleRingBuffer.java
index fa3ddcd..c7a17c4 100644
--- a/core/src/main/java/hivemall/utils/collections/DoubleRingBuffer.java
+++ b/core/src/main/java/hivemall/utils/collections/DoubleRingBuffer.java
@@ -1,19 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.utils.collections;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/utils/collections/FixedIntArray.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/utils/collections/FixedIntArray.java b/core/src/main/java/hivemall/utils/collections/FixedIntArray.java
index e0a2532..d4acdc1 100644
--- a/core/src/main/java/hivemall/utils/collections/FixedIntArray.java
+++ b/core/src/main/java/hivemall/utils/collections/FixedIntArray.java
@@ -1,19 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.utils.collections;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/utils/collections/IMapIterator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/utils/collections/IMapIterator.java b/core/src/main/java/hivemall/utils/collections/IMapIterator.java
index ea1d6d6..7e9170e 100644
--- a/core/src/main/java/hivemall/utils/collections/IMapIterator.java
+++ b/core/src/main/java/hivemall/utils/collections/IMapIterator.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.utils.collections;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/utils/collections/IndexedSet.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/utils/collections/IndexedSet.java b/core/src/main/java/hivemall/utils/collections/IndexedSet.java
index c44018b..d636b86 100644
--- a/core/src/main/java/hivemall/utils/collections/IndexedSet.java
+++ b/core/src/main/java/hivemall/utils/collections/IndexedSet.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.utils.collections;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/utils/collections/Int2FloatOpenHashTable.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/utils/collections/Int2FloatOpenHashTable.java b/core/src/main/java/hivemall/utils/collections/Int2FloatOpenHashTable.java
index 6abea89..49f27c8 100644
--- a/core/src/main/java/hivemall/utils/collections/Int2FloatOpenHashTable.java
+++ b/core/src/main/java/hivemall/utils/collections/Int2FloatOpenHashTable.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.utils.collections;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/utils/collections/Int2IntOpenHashTable.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/utils/collections/Int2IntOpenHashTable.java b/core/src/main/java/hivemall/utils/collections/Int2IntOpenHashTable.java
index 15bd38d..211157e 100644
--- a/core/src/main/java/hivemall/utils/collections/Int2IntOpenHashTable.java
+++ b/core/src/main/java/hivemall/utils/collections/Int2IntOpenHashTable.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.utils.collections;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/utils/collections/Int2LongOpenHashTable.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/utils/collections/Int2LongOpenHashTable.java b/core/src/main/java/hivemall/utils/collections/Int2LongOpenHashTable.java
index d372ffa..2c229a4 100644
--- a/core/src/main/java/hivemall/utils/collections/Int2LongOpenHashTable.java
+++ b/core/src/main/java/hivemall/utils/collections/Int2LongOpenHashTable.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.utils.collections;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/utils/collections/IntArray.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/utils/collections/IntArray.java b/core/src/main/java/hivemall/utils/collections/IntArray.java
index 7296e74..a530efe 100644
--- a/core/src/main/java/hivemall/utils/collections/IntArray.java
+++ b/core/src/main/java/hivemall/utils/collections/IntArray.java
@@ -1,19 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.utils.collections;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/utils/collections/IntArrayList.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/utils/collections/IntArrayList.java b/core/src/main/java/hivemall/utils/collections/IntArrayList.java
index 50283a3..c0d79de 100644
--- a/core/src/main/java/hivemall/utils/collections/IntArrayList.java
+++ b/core/src/main/java/hivemall/utils/collections/IntArrayList.java
@@ -1,19 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.utils.collections;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/utils/collections/IntOpenHashMap.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/utils/collections/IntOpenHashMap.java b/core/src/main/java/hivemall/utils/collections/IntOpenHashMap.java
index c692a0f..4621e6d 100644
--- a/core/src/main/java/hivemall/utils/collections/IntOpenHashMap.java
+++ b/core/src/main/java/hivemall/utils/collections/IntOpenHashMap.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.utils.collections;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/utils/collections/IntOpenHashTable.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/utils/collections/IntOpenHashTable.java b/core/src/main/java/hivemall/utils/collections/IntOpenHashTable.java
index 2455c4c..8d0cdf2 100644
--- a/core/src/main/java/hivemall/utils/collections/IntOpenHashTable.java
+++ b/core/src/main/java/hivemall/utils/collections/IntOpenHashTable.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.utils.collections;
 



[27/50] [abbrv] incubator-hivemall git commit: Do not force check on mvn install

Posted by my...@apache.org.
Do not force check on mvn install

Project: http://git-wip-us.apache.org/repos/asf/incubator-hivemall/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-hivemall/commit/67700982
Tree: http://git-wip-us.apache.org/repos/asf/incubator-hivemall/tree/67700982
Diff: http://git-wip-us.apache.org/repos/asf/incubator-hivemall/diff/67700982

Branch: refs/heads/master
Commit: 677009825c7a40cbe28b1c77222a7d667a61a912
Parents: fc97a52
Author: myui <yu...@gmail.com>
Authored: Mon Nov 7 15:05:14 2016 +0900
Committer: myui <yu...@gmail.com>
Committed: Mon Nov 7 15:05:14 2016 +0900

----------------------------------------------------------------------
 pom.xml | 7 -------
 1 file changed, 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/67700982/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 6616f47..62cc5a5 100644
--- a/pom.xml
+++ b/pom.xml
@@ -408,13 +408,6 @@
 						  <md>XML_STYLE</md>
 						</mapping>
 					</configuration>
-					<executions>
-						<execution>
-							<goals>
-								<goal>check</goal>
-							</goals>
-						</execution>
-					</executions>
 				</plugin>
 			</plugins>
 		</pluginManagement>


[10/50] [abbrv] incubator-hivemall git commit: HIVEMALL-13: Fail to building xgboost due to unbound variable HIVEMALL_HOME (#377)

Posted by my...@apache.org.
HIVEMALL-13: Fail to building xgboost due to unbound variable HIVEMALL_HOME (#377)



Project: http://git-wip-us.apache.org/repos/asf/incubator-hivemall/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-hivemall/commit/5ebdc06a
Tree: http://git-wip-us.apache.org/repos/asf/incubator-hivemall/tree/5ebdc06a
Diff: http://git-wip-us.apache.org/repos/asf/incubator-hivemall/diff/5ebdc06a

Branch: refs/heads/master
Commit: 5ebdc06aa2c7dd03f893525fc5ecfecb86659e3c
Parents: f443c06
Author: Kai Sasaki <le...@me.com>
Authored: Fri Oct 28 16:47:52 2016 +0900
Committer: Makoto YUI <yu...@gmail.com>
Committed: Fri Oct 28 16:47:52 2016 +0900

----------------------------------------------------------------------
 bin/build_xgboost.sh | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/5ebdc06a/bin/build_xgboost.sh
----------------------------------------------------------------------
diff --git a/bin/build_xgboost.sh b/bin/build_xgboost.sh
index 7b97eea..43ea515 100755
--- a/bin/build_xgboost.sh
+++ b/bin/build_xgboost.sh
@@ -17,7 +17,7 @@
 # See the License for the specific language governing permissions and
 # limitations under the License.
 
-set -eu
+set -e
 set -o pipefail
 
 # Target commit hash value


[02/50] [abbrv] incubator-hivemall git commit: Added a gitbook userguide

Posted by my...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/370e2aa3/docs/gitbook/recommend/item_based_cf.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/recommend/item_based_cf.md b/docs/gitbook/recommend/item_based_cf.md
new file mode 100644
index 0000000..a4a8cfd
--- /dev/null
+++ b/docs/gitbook/recommend/item_based_cf.md
@@ -0,0 +1,565 @@
+This document describe how to do Item-based Collaborative Filtering using Hivemall.
+
+_Caution: naive similarity computation is `O(n^2)` to compute all item-item pair similarity. [MinHash](https://en.wikipedia.org/wiki/MinHash#Jaccard_similarity_and_minimum_hash_values) is an efficient scheme for computing jaccard similarity. Section 6 show how to use MinHash in Hivemall._
+
+## 1. Prepare transaction table
+
+Prepare following transaction table. We are generating `feature_vector` for each `item_id` based on cooccurrence of purchased items, a sort of bucket analysis.
+
+| userid | itemid | purchase_at `timestamp` |
+|:-:|:-:|:-:| 
+| 1 | 31231 | 2015-04-9 00:29:02 |
+| 1 | 13212 | 2016-05-24 16:29:02 |
+| 2 | 312 | 2016-06-03 23:29:02 |
+| 3 | 2313 | 2016-06-04 19:29:02 |
+| .. | .. | .. |
+
+## 2. Create item_features table
+
+What we want for creating a feature vector for each item is the following `cooccurrence` relation.
+
+| itemid | other | cnt |
+|:-:|:-:|:-:|
+| 583266 | 621056 | 9999 |
+| 583266 | 583266 | 18 |
+| 31231 | 13212 | 129 |
+| 31231 | 31231 | 3 |
+| 31231	| 9833 | 953 |
+| ... | ... | ... |
+
+Feature vectors of each item will be as follows:
+
+| itemid | feature_vector `array<string>` |
+|:-:|:-:|
+| 583266 | 621056:9999, 583266:18 |
+| 31231 | 13212:129, 31231:3, 9833:953 |
+| ... | ... |
+
+Note that value of feature vector should be scaled for k-NN similarity computation e.g., as follows:
+
+| itemid | feature_vector `array<string>` |
+|:-:|:-:|
+| 583266 | 621056:`ln(9999+1)`, 583266:`ln(18+1)` |
+| 31231 | 13212:`ln(129+1)`, 31231:`ln(3+1)`, 9833:`ln(953+1)` |
+| ... | ... |
+
+The following queries results in creating the above table.
+
+### 2.1. Creating Item purchased table
+
+The following query creates a table that contains userid, itemid, and purchased_at. The table represents the last user-item contact (purchase) while the `transaction` table holds all contacts.
+
+```sql
+CREATE TABLE user_purchased as
+-- INSERT OVERWRITE TABLE user_purchased
+select 
+  userid,
+  itemid,
+  max(purchased_at) as purchased_at,
+  count(1) as purchase_count
+from
+  transaction
+-- where purchased_at < xxx -- divide training/testing data by time 
+group by
+  userid, itemid
+;
+```
+
+**Note:** _Better to avoid too old transactions because those information would be outdated though an enough number of transactions is required for recommendation._
+
+### 2.2. Creating cooccurrence table
+
+**Caution:** _Item-Item cooccurrence matrix is a symmetric matrix that has the number of total occurrence for each diagonal element . If the size of items are `k`, then the size of expected matrix is `k * (k - 1) / 2`, usually a very large one._
+
+_Better to use [2.2.2.](https://github.com/myui/hivemall/wiki/Item-based-Collaborative-Filtering#limiting-size-of-elements-in-cooccurrence_upper_triangular) instead of [2.2.1.](https://github.com/myui/hivemall/wiki/Item-based-Collaborative-Filtering#221-create-cooccurrence-table-directly) for creating a `cooccurrence` table where dataset is large._
+
+### 2.2.1. Create cooccurrence table directly
+
+```sql
+create table cooccurrence as 
+-- INSERT OVERWRITE TABLE cooccurrence
+select
+  u1.itemid,
+  u2.itemid as other, 
+  count(1) as cnt
+from
+  user_purchased u1
+  JOIN user_purchased u2 ON (u1.userid = u2.userid)
+where
+  u1.itemid != u2.itemid 
+  -- AND u2.purchased_at >= u1.purchased_at -- the other item should be purchased with/after the base item
+group by
+  u1.itemid, u2.itemid
+-- having -- optional but recommended to have this condition where dataset is large
+--  cnt >= 2 -- count(1) >= 2
+;
+```
+
+**Caution:** Note that specifying `having cnt >= 2` has a drawback that item cooccurrence is not calculated where `cnt` is less than 2. It could result no recommended items for certain items. Please ignore `having cnt >= 2` if the following computations finish in an acceptable/reasonable time.
+
+**Caution:** _We ignore a purchase order in the following example. It means that the occurrence counts of `ItemA -> ItemB` and `ItemB -> ItemA` are assumed to be same. It is sometimes not a good idea e.g., for `Camera -> SD card` and `SD card -> Camera`._
+
+### 2.2.2. Create cooccurrence table from Upper Triangular Matrix of cooccurrence
+
+Better to create [Upper Triangular Matrix](https://en.wikipedia.org/wiki/Triangular_matrix#Description) that has `itemid > other` if resulting table is very large. No need to create Upper Triangular Matrix if your Hadoop cluster can handle the following instructions without considering it.
+
+```sql
+create table cooccurrence_upper_triangular as 
+-- INSERT OVERWRITE TABLE cooccurrence_upper_triangular
+select
+  u1.itemid,
+  u2.itemid as other, 
+  count(1) as cnt
+from
+  user_purchased u1
+  JOIN user_purchased u2 ON (u1.userid = u2.userid)
+where
+  u1.itemid > u2.itemid 
+group by
+  u1.itemid, u2.itemid
+;
+```
+
+```sql
+create table cooccurrence as 
+-- INSERT OVERWRITE TABLE cooccurrence
+select * from (
+  select itemid, other, cnt from cooccurrence_upper_triangular
+  UNION ALL
+  select other as itemid, itemid as other, cnt from cooccurrence_upper_triangular
+) t; 
+```
+
+_Note: `UNION ALL` [required to be embedded](https://cwiki.apache.org/confluence/display/Hive/LanguageManual+Union#LanguageManualUnion-UNIONwithinaFROMClause) in Hive._
+
+### Limiting size of elements in cooccurrence_upper_triangular
+
+```sql
+create table cooccurrence_upper_triangular as
+WITH t1 as (
+  select
+    u1.itemid,
+    u2.itemid as other, 
+    count(1) as cnt
+  from
+    user_purchased u1
+    JOIN user_purchased u2 ON (u1.userid = u2.userid)
+  where
+    u1.itemid > u2.itemid 
+  group by
+    u1.itemid, u2.itemid
+),
+t2 as (
+  select
+    each_top_k( -- top 1000
+      1000, itemid, cnt, 
+      itemid, other, cnt
+    ) as (rank, cmpkey, itemid, other, cnt)
+  from (
+    select * from t1
+    CLUSTER BY itemid
+  ) t;
+)
+-- INSERT OVERWRITE TABLE cooccurrence_upper_triangular
+select itemid, other, cnt
+from t2;
+```
+
+```sql
+create table cooccurrence as 
+WITh t1 as (
+  select itemid, other, cnt from cooccurrence_upper_triangular
+  UNION ALL
+  select other as itemid, itemid as other, cnt from cooccurrence_upper_triangular
+),
+t2 as (
+  select
+    each_top_k(
+      1000, itemid, cnt,
+      itemid, other, cnt
+    ) as (rank, cmpkey, itemid, other, cnt)
+  from (
+    select * from t1
+    CLUSTER BY itemid
+  ) t
+)
+-- INSERT OVERWRITE TABLE cooccurrence
+select itemid, other, cnt
+from t2;
+```
+
+### 2.2.3. Computing cooccurrence ratio (optional step)
+
+You can optionally compute cooccurrence ratio as follows:
+
+```sql
+WITH stats as (
+  select 
+    itemid,
+    sum(cnt) as totalcnt
+  from 
+    cooccurrence
+  group by
+    itemid
+)
+INSERT OVERWRITE TABLE cooccurrence_ratio
+SELECT
+  l.itemid,
+  l.other, 
+  (l.cnt / r.totalcnt) as ratio
+FROM
+  cooccurrence l
+  JOIN stats r ON (l.itemid = r.itemid)
+group by
+  l.itemid, l.other
+;
+```
+
+`l.cnt / r.totalcnt` represents a cooccurrence ratio of range `[0,1]`.
+
+### 2.3. creating a feature vector for each item
+
+```sql
+INSERT OVERWRITE TABLE item_features
+SELECT
+  itemid,
+  -- scaling `ln(cnt+1)` to avoid large value in the feature vector
+  -- rounding to xxx.yyyyy to reduce size of feature_vector in array<string>
+  collect_list(feature(other, round(ln(cnt+1),5))) as feature_vector
+FROM
+  cooccurrence
+GROUP BY
+  itemid
+;
+```
+
+## 3. Computing Item similarity scores
+
+Item-Item similarity computation is known to be computation complexity `O(n^2)` where `n` is the number of items.
+Depending on your cluster size and your dataset, the optimal solution differs.
+
+**Note:** _Better to use [3.1.1.](https://github.com/myui/hivemall/wiki/Item-based-Collaborative-Filtering#311-similarity-computation-using-the-symmetric-property-of-item-similarity-matrix) scheme where dataset is large._
+
+### 3.1. Shuffle heavy similarity computation
+
+This version involves 3-way joins w/ large data shuffle; However, this version works in parallel where a cluster has enough task slots.
+
+```sql
+WITH similarity as (
+  select
+    o.itemid,
+    o.other,
+    cosine_similarity(t1.feature_vector, t2.feature_vector) as similarity
+  from
+    cooccurrence o
+    JOIN item_features t1 ON (o.itemid = t1.itemid)
+    JOIN item_features t2 ON (o.other = t2.itemid)
+),
+topk as (
+  select
+    each_top_k( -- get top-10 items based on similarity score
+      10, itemid, similarity,
+      itemid, other -- output items
+    ) as (rank, similarity, itemid, other)
+  from (
+    select * from similarity
+    where similarity > 0 -- similarity > 0.01
+    CLUSTER BY itemid
+  ) t
+)
+INSERT OVERWRITE TABLE item_similarity
+select 
+  itemid, other, similarity
+from 
+  topk;
+```
+
+### 3.1.1. Similarity computation using the symmetric property of Item similarity matrix
+
+Note `item_similarity` is a similarity matrix. So, you can compute it from an upper triangular matrix as follows.
+
+```sql
+WITH cooccurrence_top100 as (
+  select
+    each_top_k(
+      100, itemid, cnt,  
+      itemid, other
+    ) as (rank, cmpkey, itemid, other)
+  from (
+    select * from cooccurrence_upper_triangular
+    CLUSTER BY itemid
+  ) t
+), 
+similarity as (
+  select
+    o.itemid,
+    o.other,
+    cosine_similarity(t1.feature_vector, t2.feature_vector) as similarity
+  from
+    cooccurrence_top100 o\u3000
+    -- cooccurrence_upper_triangular o
+    JOIN item_features t1 ON (o.itemid = t1.itemid)
+    JOIN item_features t2 ON (o.other = t2.itemid)
+),
+topk as (
+  select
+    each_top_k( -- get top-10 items based on similarity score
+      10, itemid, similarity,
+      itemid, other -- output items
+    ) as (rank, similarity, itemid, other)
+  from (
+    select * from similarity
+    where similarity > 0 -- similarity > 0.01
+    CLUSTER BY itemid
+  ) t
+)
+INSERT OVERWRITE TABLE item_similarity_upper_triangler
+select 
+  itemid, other, similarity
+from 
+  topk;
+```
+
+```sql
+INSERT OVERWRITE TABLE item_similarity
+select * from (
+  select itemid, other, similarity from item_similarity_upper_triangler
+  UNION ALL
+  select other as itemid, itemid as other, similarity from item_similarity_upper_triangler
+) t;
+```
+
+### 3.2. Computation heavy similarity computation 
+
+Alternatively, you can compute cosine similarity as follows. This version involves cross join and thus runs sequentially in a single task. However, it involves less shuffle when compared to 3.1.
+
+```sql
+WITH similarity as (
+  select
+   t1.itemid,
+   t2.itemid as other,
+   cosine_similarity(t1.feature_vector, t2.feature_vector) as similarity
+  from
+   item_features t1
+   CROSS JOIN item_features t2
+  WHERE
+    t1.itemid != t2.itemid
+),
+topk as (
+  select
+    each_top_k( -- get top-10 items based on similarity score
+      10, itemid, similarity,
+      itemid, other -- output items
+    ) as (rank, similarity, itemid, other)
+  from (
+    select * from similarity
+    where similarity > 0 -- similarity > 0.01
+    CLUSTER BY itemid
+  ) t
+)
+INSERT OVERWRITE TABLE item_similarity
+select 
+  itemid, other, similarity
+from 
+  topk
+;
+```
+
+| item | other | similarity |
+|:-:|:-:|:-:|
+| 583266 | 621056 | 0.33 |
+| 583266 | 583266 | 0.18 |
+| 31231 | 13212 | 1.29 |
+| 31231 | 31231 | 0.3 |
+| 31231	| 9833 | 0.953 |
+| ... | ... | ... |
+
+## 4. Item-based Recommendation
+
+This section introduces item-based recommendation based on recently purchased items by each user.
+
+**Caution:** _It would better to ignore recommending some of items that user already purchased (only 1 time) while items that are purchased twice or more would be okey to be included in the recommendation list (e.g., repeatedly purchased daily necessities). So, you would need an item property table showing that each item is repeatedly purchased items or not._
+
+### 4.1. Computes top-k recently purchaed items for each user
+
+First, prepare `recently_purchased_items` table as follows:
+
+```sql
+INSERT OVERWRITE TABLE recently_purchased_items
+select
+  each_top_k( -- get top-5 recently purchased items for each user
+     5, userid, purchased_at,
+     userid, itemid
+  ) as (rank, purchased_at, userid, itemid)
+from (
+  select
+    purchased_at, userid, itemid
+  from 
+    user_purchased
+  -- where [optional filtering]
+  --  purchased_at >= xxx -- divide training/test data by time
+  CLUSTER BY
+    user_id -- Note CLUSTER BY is mandatory when using each_top_k
+) t;
+```
+
+### 4.2. Recommend top-k items based on the cooccurrence for each user's recently purchased item
+
+```sql
+WITH topk as (
+  select
+    each_top_k(
+       5, userid, cnt,
+       userid, other
+    ) as (rank, cnt, userid, rec_item)
+  from (
+    select 
+      t1.userid, t2.other, max(t2.cnt) as cnt
+    from
+      recently_purchased_items t1
+      JOIN cooccurrence t2 ON (t1.itemid = t2.itemid)
+    where
+      t1.itemid != t2.other -- do not include items that user already purchased
+      AND NOT EXISTS (
+        SELECT a.itemid FROM user_purchased a
+        WHERE a.userid = t1.userid AND a.itemid = t2.other
+--        AND a.purchased_count <= 1 -- optional
+      )
+    group by
+      t1.userid, t2.other
+    CLUSTER BY
+      userid -- top-k grouping by userid
+  ) t1
+)
+INSERT OVERWRITE TABLE item_recommendation
+select
+  userid,
+  map_values(to_ordered_map(rank, rec_item)) as rec_items
+from
+  topk
+group by
+  userid
+;
+```
+
+### 4.3. Recommend top-k items based on the (cooccurrence) similarity for each user's recently purchased item
+
+```sql
+WITH topk as (
+  select
+    each_top_k(
+       5, userid, similarity,
+       userid, other
+    ) as (rank, similarity, userid, rec_item)
+  from (
+    select
+      t1.userid, t2.other, max(t2.similarity) as similarity
+    from
+      recently_purchased_items t1
+      JOIN item_similarity t2 ON (t1.itemid = t2.itemid)
+    where
+      t1.itemid != t2.other -- do not include items that user already purchased
+      AND NOT EXISTS (
+        SELECT a.itemid FROM user_purchased a
+        WHERE a.userid = t1.userid AND a.itemid = t2.other
+--        AND a.purchased_count <= 1 -- optional
+      )
+    group by
+      t1.userid, t2.other
+    CLUSTER BY
+      userid -- top-k grouping by userid
+  ) t1
+)
+INSERT OVERWRITE TABLE item_recommendation
+select
+  userid,
+  map_values(to_ordered_map(rank, rec_item)) as rec_items
+from
+  topk
+group by
+  userid
+;
+```
+
+## 5. Pseudo Jaccard Similarity computation using MinHash
+
+Refer [this article](https://en.wikipedia.org/wiki/MinHash#Jaccard_similarity_and_minimum_hash_values
+) to get details about MinHash and Jarccard similarity. [This blog article](https://blog.treasuredata.com/blog/2016/02/16/minhash-in-hivemall/) also explains about Hivemall's minhash.
+
+```sql
+INSERT OVERWRITE TABLE minhash -- results in 30x records of item_features
+select  
+  -- assign 30 minhash values for each item
+  minhash(itemid, feature_vector, "-n 30") as (clusterid, itemid) -- '-n' would be 10~100
+from
+  item_features
+;
+
+WITH t1 as (
+  select
+    l.itemid,
+    r.itemid as other,
+    count(1) / 30 as similarity -- Pseudo jaccard similarity '-n 30'
+  from
+    minhash l 
+    JOIN minhash r 
+      ON (l.clusterid = r.clusterid)
+  where 
+    l.itemid != r.itemid
+  group by
+    l.itemid, r.itemid
+  having
+    count(1) >= 3 -- [optional] filtering equals to (count(1)/30) >= 0.1
+),
+top100 as (
+  select
+    each_top_k(100, itemid, similarity, itemid, other)
+      as (rank, similarity, itemid, other)
+  from (
+    select * from t1 
+    -- where similarity >= 0.1 -- Optional filtering. Can be ignored.
+    CLUSTER BY itemid 
+  ) t2
+)
+INSERT OVERWRITE TABLE jaccard_similarity
+select
+  itemid, other, similarity
+from
+  top100
+;
+```
+_Caution: Note that there might be no similar item for certain items._
+
+### 5.1. Cosine similarity computation following minhash-based similarity items filtering
+
+You can compute `top-k` similar items based on cosine similarity, following rough `top-N` similar items listing using minhash, where `k << N` (e.g., k=10 and N=100).
+
+```sql
+WITH similarity as (
+  select
+    o.itemid,
+    o.other,
+    cosine_similarity(t1.feature_vector, t2.feature_vector) as similarity
+  from
+    jaccard_similarity o
+    JOIN item_features t1 ON (o.itemid = t1.itemid)
+    JOIN item_features t2 ON (o.other = t2.itemid)
+),
+topk as (
+  select
+    each_top_k( -- get top-10 items based on similarity score
+      10, itemid, similarity,
+      itemid, other -- output items
+    ) as (rank, similarity, itemid, other)
+  from (
+    select * from similarity
+    where similarity > 0 -- similarity > 0.01
+    CLUSTER BY itemid
+  ) t
+)
+INSERT OVERWRITE TABLE cosine_similarity
+select 
+  itemid, other, similarity
+from 
+  topk;
+```
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/370e2aa3/docs/gitbook/recommend/movielens.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/recommend/movielens.md b/docs/gitbook/recommend/movielens.md
new file mode 100644
index 0000000..e69de29

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/370e2aa3/docs/gitbook/recommend/movielens_cv.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/recommend/movielens_cv.md b/docs/gitbook/recommend/movielens_cv.md
new file mode 100644
index 0000000..ec2255b
--- /dev/null
+++ b/docs/gitbook/recommend/movielens_cv.md
@@ -0,0 +1,63 @@
+[Cross-validation](http://en.wikipedia.org/wiki/Cross-validation_(statistics)#k-fold_cross-validationk-fold cross validation) is a model validation technique for assessing how a prediction model will generalize to an independent data set. This example shows a way to perform [k-fold cross validation](http://en.wikipedia.org/wiki/Cross-validation_(statistics)#k-fold_cross-validation) to evaluate prediction performance.
+
+*Caution:* Matrix factorization is supported in Hivemall v0.3 or later.
+
+# Data set creating for 10-folds cross validation.
+```sql
+use movielens;
+
+set hivevar:kfold=10;
+set hivevar:seed=31;
+
+-- Adding group id (gid) to each training instance
+drop table ratings_groupded;
+create table ratings_groupded
+as
+select
+  rand_gid2(${kfold}, ${seed}) gid, -- generates group id ranging from 1 to 10
+  userid, 
+  movieid, 
+  rating
+from
+  ratings
+cluster by gid, rand(${seed});
+```
+
+## Set training hyperparameters
+
+```sql
+-- latent factors
+set hivevar:factor=10;
+-- maximum number of iterations
+set hivevar:iters=50;
+-- regularization parameter
+set hivevar:lambda=0.05;
+-- learning rate
+set hivevar:eta=0.005;
+-- conversion rate (if changes between iterations became less or equals to ${cv_rate}, the training will stop)
+set hivevar:cv_rate=0.001;
+```
+_Due to [a bug](https://issues.apache.org/jira/browse/HIVE-8396) in Hive, do not issue comments in CLI._
+
+```sql
+select avg(rating) from ratings;
+```
+> 3.581564453029317
+
+```sql
+-- mean rating value (Optional but recommended to set ${mu})
+set hivevar:mu=3.581564453029317;
+```
+_Note that it is not necessary to set an exact value for ${mu}._
+
+## SQL-generation for 10-folds cross validation
+
+Run [generate_cv.sh](https://gist.github.com/myui/c2009e5791cca650a4d0) and create [generate_cv.sql](https://gist.github.com/myui/2e2018217e2188222655).
+
+Then, issue SQL queies in [generate_cv.sql](https://gist.github.com/myui/2e2018217e2188222655) to get MAE/RMSE.
+
+> 0.6695442192077673 (MAE)
+
+> 0.8502739040257945 (RMSE)
+
+_We recommend to use [Tez](http://tez.apache.org/) for running queries having many stages._
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/370e2aa3/docs/gitbook/recommend/movielens_dataset.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/recommend/movielens_dataset.md b/docs/gitbook/recommend/movielens_dataset.md
new file mode 100644
index 0000000..f175f4f
--- /dev/null
+++ b/docs/gitbook/recommend/movielens_dataset.md
@@ -0,0 +1,160 @@
+# Data preparation
+
+First, downlod MovieLens dataset from the following site.
+> http://www.grouplens.org/system/files/ml-1m.zip
+
+Get detail about the dataset in the README.
+> http://files.grouplens.org/papers/ml-1m-README.txt
+
+You can find three dat file in the archive: 
+> movies.dat, ratings.dat, users.dat.
+
+Change column separator as follows:
+```sh
+sed 's/::/#/g' movies.dat > movies.t
+sed 's/::/#/g' ratings.dat > ratings.t
+sed 's/::/#/g' users.dat > users.t
+```
+
+Create a file named occupations.t with the following contents:
+```
+0#other/not specified
+1#academic/educator
+2#artist
+3#clerical/admin
+4#college/grad student
+5#customer service
+6#doctor/health care
+7#executive/managerial
+8#farmer
+9#homemaker
+10#K-12 student
+11#lawyer
+12#programmer
+13#retired
+14#sales/marketing
+15#scientist
+16#self-employed
+17#technician/engineer
+18#tradesman/craftsman
+19#unemployed
+20#writer
+```
+
+# Importing data as Hive tables
+
+## create tables
+```sql
+create database movielens;
+use movielens;
+
+CREATE EXTERNAL TABLE ratings (
+  userid INT, 
+  movieid INT,
+  rating INT, 
+  tstamp STRING
+) ROW FORMAT DELIMITED
+FIELDS TERMINATED BY '#'
+STORED AS TEXTFILE
+LOCATION '/dataset/movielens/ratings';
+
+CREATE EXTERNAL TABLE movies (
+  movieid INT, 
+  title STRING,
+  genres ARRAY<STRING>
+) ROW FORMAT DELIMITED
+FIELDS TERMINATED BY '#'
+COLLECTION ITEMS TERMINATED BY "|"
+STORED AS TEXTFILE
+LOCATION '/dataset/movielens/movies';
+
+CREATE EXTERNAL TABLE users (
+  userid INT, 
+  gender STRING, 
+  age INT,
+  occupation INT,
+  zipcode STRING
+) ROW FORMAT DELIMITED
+FIELDS TERMINATED BY '#'
+STORED AS TEXTFILE
+LOCATION '/dataset/movielens/users';
+
+CREATE EXTERNAL TABLE occupations (
+  id INT,
+  occupation STRING
+) ROW FORMAT DELIMITED
+FIELDS TERMINATED BY '#'
+STORED AS TEXTFILE
+LOCATION '/dataset/movielens/occupations';
+```
+
+## load data into tables
+```sh
+hadoop fs -put ratings.t /dataset/movielens/ratings
+hadoop fs -put movies.t /dataset/movielens/movies
+hadoop fs -put users.t /dataset/movielens/users
+hadoop fs -put occupations.t /dataset/movielens/occupations
+```
+
+# Create a concatenated table 
+```sql
+CREATE TABLE rating_full
+as
+select 
+  r.*, 
+  m.title as m_title,
+  concat_ws('|',sort_array(m.genres)) as m_genres, 
+  u.gender as u_gender,
+  u.age as u_age,
+  u.occupation as u_occupation,
+  u.zipcode as u_zipcode
+from
+  ratings r 
+  JOIN movies m ON (r.movieid = m.movieid)
+  JOIN users u ON (r.userid = u.userid);
+```
+
+hive> desc rating_full;
+```
+userid                  int                     None
+movieid                 int                     None
+rating                  int                     None
+tstamp                  string                  None
+m_title                 string                  None
+m_genres                string                  None
+u_gender                string                  None
+u_age                   int                     None
+u_occupation            int                     None
+u_zipcode               string                  None
+```
+
+---
+# Creating training/testing data
+
+Create a training/testing table such that each has 80%/20% of the original rating data.
+
+```sql
+-- Adding rowids to the rating table
+SET hivevar:seed=31;
+CREATE TABLE ratings2
+as
+select
+  rand(${seed}) as rnd, 
+  userid, 
+  movieid, 
+  rating
+from 
+  ratings;
+
+CREATE TABLE training
+as
+select * from ratings2
+order by rnd DESC
+limit 800000;
+
+CREATE TABLE testing
+as
+select * from ratings2
+order by rnd ASC
+limit 200209;
+```

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/370e2aa3/docs/gitbook/recommend/movielens_fm.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/recommend/movielens_fm.md b/docs/gitbook/recommend/movielens_fm.md
new file mode 100644
index 0000000..282e923
--- /dev/null
+++ b/docs/gitbook/recommend/movielens_fm.md
@@ -0,0 +1,249 @@
+_Caution: Factorization Machine is supported from Hivemall v0.4 or later._
+
+# Data preparation
+
+First of all, please create `ratings` table described in the following page: 
+https://github.com/myui/hivemall/wiki/MovieLens-Dataset
+
+```sql
+use movielens;
+
+SET hivevar:seed=31;
+
+DROP TABLE ratings_fm;
+CREATE TABLE ratings_fm
+as
+select
+	rowid() as rowid,
+	categorical_features(array("userid","movieid"), userid, movieid) 
+	  as features,
+	rating,
+	rand(${seed}) as rnd
+from
+	ratings
+CLUSTER BY rand(43); -- shuffle training input
+
+select * from ratings_fm limit 2;
+```
+
+| rowid | features | rating | rnd |
+|:-----:|:--------:|:------:|:---:|
+| 1-383970 | ["userid#2244","movieid#1272"] | 5 | 0.33947035987020546 |
+| 1-557913 | ["userid#3425","movieid#2791"] | 4 | 0.12344886396954391 |
+
+```sql
+-- use 80% for training
+DROP TABLE training_fm;
+CREATE TABLE training_fm
+as
+select * from ratings_fm
+order by rnd DESC
+limit 800000;
+
+-- use 20% for testing
+DROP TABLE testing_fm;
+CREATE TABLE testing_fm
+as
+select * from ratings_fm
+order by rnd ASC
+limit 200209;
+
+-- testing table for prediction
+CREATE OR REPLACE VIEW testing_fm_exploded
+as 
+select 
+  rowid,
+  extract_feature(fv) as feature,
+  extract_weight(fv) as Xi,
+  rating
+from
+  testing_fm t1 LATERAL VIEW explode(add_bias(features)) t2 as fv;
+```
+_Caution: Don't forget to call `add_bias` in the above query. No need to call `add_bias` for preparing training data in Factorization Machines because it always considers it._
+
+# Training
+
+## Hyperparamters for Training
+```sql
+-- number of factors
+set hivevar:factor=10;
+-- maximum number of training iterations
+set hivevar:iters=50;
+```
+
+## Build a prediction mdoel by Factorization Machine
+
+```sql
+drop table fm_model;
+create table fm_model
+as
+select
+  feature,
+  avg(Wi) as Wi,
+  array_avg(Vif) as Vif
+from (
+  select 
+    train_fm(features, rating, "-factor ${factor} -iters ${iters} -eta 0.01") 
+    	as (feature, Wi, Vif)
+  from 
+    training_fm
+) t
+group by feature;
+```
+
+_Note: setting eta option is optional. However, setting `-eta 0.01` usually works well._
+
+## Usage of `train_fm`
+
+You can get usages of `train_fm` by giving `-help` option as follows:
+```sql
+select 
+  train_fm(features, rating, "-help") as (feature, Wi, Vif)
+from 
+  training_fm
+```
+
+```
+usage: train_fm(array<string> x, double y [, const string options]) -
+       Returns a prediction value [-adareg] [-c] [-cv_rate <arg>]
+       [-disable_cv] [-eta <arg>] [-eta0 <arg>] [-f <arg>] [-help]
+       [-init_v <arg>] [-int_feature] [-iters <arg>] [-lambda <arg>] [-max
+       <arg>] [-maxval <arg>] [-min <arg>] [-min_init_stddev <arg>] [-p
+       <arg>] [-power_t <arg>] [-seed <arg>] [-sigma <arg>] [-t <arg>]
+       [-va_ratio <arg>] [-va_threshold <arg>]
+ -adareg,--adaptive_regularizaion             Whether to enable adaptive
+                                              regularization [default:
+                                              OFF]
+ -c,--classification                          Act as classification
+ -cv_rate,--convergence_rate <arg>            Threshold to determine
+                                              convergence [default: 0.005]
+ -disable_cv,--disable_cvtest                 Whether to disable
+                                              convergence check [default:
+                                              OFF]
+ -eta <arg>                                   The initial learning rate
+ -eta0 <arg>                                  The initial learning rate
+                                              [default 0.1]
+ -f,--factor <arg>                            The number of the latent
+                                              variables [default: 10]
+ -help                                        Show function help
+ -init_v <arg>                                Initialization strategy of
+                                              matrix V [random, gaussian]
+                                              (default: random)
+ -int_feature,--feature_as_integer            Parse a feature as integer
+                                              [default: OFF, ON if -p
+                                              option is specified]
+ -iters,--iterations <arg>                    The number of iterations
+                                              [default: 1]
+ -lambda,--lambda0 <arg>                      The initial lambda value for
+                                              regularization [default:
+                                              0.01]
+ -max,--max_target <arg>                      The maximum value of target
+                                              variable
+ -maxval,--max_init_value <arg>               The maximum initial value in
+                                              the matrix V [default: 1.0]
+ -min,--min_target <arg>                      The minimum value of target
+                                              variable
+ -min_init_stddev <arg>                       The minimum standard
+                                              deviation of initial matrix
+                                              V [default: 0.1]
+ -p,--size_x <arg>                            The size of x
+ -power_t <arg>                               The exponent for inverse
+                                              scaling learning rate
+                                              [default 0.1]
+ -seed <arg>                                  Seed value [default: -1
+                                              (random)]
+ -sigma <arg>                                 The standard deviation for
+                                              initializing V [default:
+                                              0.1]
+ -t,--total_steps <arg>                       The total number of training
+                                              examples
+ -va_ratio,--validation_ratio <arg>           Ratio of training data used
+                                              for validation [default:
+                                              0.05f]
+ -va_threshold,--validation_threshold <arg>   Threshold to start
+                                              validation. At least N
+                                              training examples are used
+                                              before validation [default:
+                                              1000]
+```
+
+# Prediction
+
+```sql
+-- workaround for a bug 
+-- https://github.com/myui/hivemall/wiki/Map-side-Join-causes-ClassCastException-on-Tez:-LazyBinaryArray-cannot-be-cast-to-%5BLjava.lang.Object;
+set hive.mapjoin.optimized.hashtable=false;
+
+drop table fm_predict;
+create table fm_predict
+as
+select
+  t1.rowid,
+  fm_predict(p1.Wi, p1.Vif, t1.Xi) as predicted
+from 
+  testing_fm_exploded t1
+  LEFT OUTER JOIN fm_model p1 ON (t1.feature = p1.feature)
+group by
+  t1.rowid;
+```
+
+# Evaluation
+
+```sql
+select
+  mae(p.predicted, rating) as mae,
+  rmse(p.predicted, rating) as rmse
+from
+  testing_fm as t
+  JOIN fm_predict as p on (t.rowid = p.rowid);
+```
+
+> 0.6736798239047873 (mae)     0.858938110314545 (rmse)
+
+# Fast Factorization Machines Training using Int Features
+
+Training of Factorization Machines (FM) can be done more efficietly, in term of speed, by using INT features.
+In this section, we show how to run FM training by using int features, more specifically by using [feature hashing](https://github.com/myui/hivemall/wiki/Feature-hashing).
+
+```sql
+set hivevar:factor=10;
+set hivevar:iters=50;
+
+drop table fm_model;
+create table fm_model
+as
+select
+  feature,
+  avg(Wi) as Wi,
+  array_avg(Vif) as Vif
+from (
+  select 
+    train_fm(feature_hashing(features), rating, "-factor ${factor} -iters ${iters} -eta 0.01 -int_feature")  -- internally use a sparse map
+ -- train_fm(feature_hashing(features), rating, "-factor ${factor} -iters ${iters} -eta 0.01 -int_feature -num_features 16777216") -- internally use a dense array 
+        as (feature, Wi, Vif)
+  from 
+    training_fm
+) t
+group by feature;
+```
+
+```sql
+set hive.mapjoin.optimized.hashtable=false; -- workaround for https://issues.apache.org/jira/browse/HIVE-11051
+
+WITH predicted as (
+  select
+    t1.rowid,
+    fm_predict(p1.Wi, p1.Vif, t1.Xi) as predicted
+  from 
+    testing_fm_exploded t1
+    LEFT OUTER JOIN fm_model p1 ON (feature_hashing(t1.feature) = p1.feature)
+  group by
+    t1.rowid
+)
+select
+  mae(p.predicted, rating) as mae,
+  rmse(p.predicted, rating) as rmse
+from
+  testing_fm as t
+  JOIN predicted as p on (t.rowid = p.rowid);
+```
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/370e2aa3/docs/gitbook/recommend/movielens_mf.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/recommend/movielens_mf.md b/docs/gitbook/recommend/movielens_mf.md
new file mode 100644
index 0000000..bc5c641
--- /dev/null
+++ b/docs/gitbook/recommend/movielens_mf.md
@@ -0,0 +1,137 @@
+This page explains how to run matrix factorization on [MovieLens 1M dataset](https://github.com/myui/hivemall/wiki/MovieLens-Dataset).
+
+*Caution:* Matrix factorization is supported in Hivemall v0.3 or later.
+
+## Calculate the mean rating in the training dataset
+```sql
+use movielens;
+
+select avg(rating) from training;
+```
+> 3.593565
+
+## Set variables (hyperparameters) for training
+```sql
+-- mean rating
+set hivevar:mu=3.593565;
+-- number of factors
+set hivevar:factor=10;
+-- maximum number of training iterations
+set hivevar:iters=50;
+```
+See [this article](https://github.com/myui/hivemall/wiki/List-of-parameters-of-Matrix-Factorization) or [OnlineMatrixFactorizationUDTF#getOption()](https://github.com/myui/hivemall/blob/master/src/main/java/hivemall/mf/OnlineMatrixFactorizationUDTF.java#L123) to get the details of options.
+
+Note that there are no need to set an exact value for $mu. It actually works without setting $mu but recommended to set one for getting a better prediction.
+
+_Due to [a bug](https://issues.apache.org/jira/browse/HIVE-8396) in Hive, do not issue comments in CLI._
+
+## Training
+```sql
+create table sgd_model
+as
+select
+  idx, 
+  array_avg(u_rank) as Pu, 
+  array_avg(m_rank) as Qi, 
+  avg(u_bias) as Bu, 
+  avg(m_bias) as Bi
+from (
+  select 
+    train_mf_sgd(userid, movieid, rating, "-factor ${factor} -mu ${mu} -iter ${iters}") as (idx, u_rank, m_rank, u_bias, m_bias)
+  from 
+    training
+) t
+group by idx;
+```
+Note: Hivemall also provides *train_mf_adagrad* for training using AdaGrad.
+
+# Predict
+
+```sql
+select
+  t2.actual,
+  mf_predict(t2.Pu, p2.Qi, t2.Bu, p2.Bi, ${mu}) as predicted
+from (
+  select
+    t1.userid, 
+    t1.movieid,
+    t1.rating as actual,
+    p1.Pu,
+    p1.Bu
+  from
+    testing t1 LEFT OUTER JOIN sgd_model p1
+    ON (t1.userid = p1.idx) 
+) t2 
+LEFT OUTER JOIN sgd_model p2
+ON (t2.movieid = p2.idx);
+```
+
+# Evaluate (computes MAE and RMSE)
+```sql
+select
+  mae(predicted, actual) as mae,
+  rmse(predicted, actual) as rmse
+from (
+  select
+    t2.actual,
+    mf_predict(t2.Pu, p2.Qi, t2.Bu, p2.Bi, ${mu}) as predicted
+  from (
+    select
+      t1.userid, 
+      t1.movieid,
+      t1.rating as actual,
+      p1.Pu,
+      p1.Bu
+    from
+      testing t1 LEFT OUTER JOIN sgd_model p1
+      ON (t1.userid = p1.idx) 
+  ) t2 
+  LEFT OUTER JOIN sgd_model p2
+  ON (t2.movieid = p2.idx)
+) t;
+```
+> 0.6728969407733578 (MAE) 
+
+> 0.8584162122694449 (RMSE)
+
+# Item Recommendation
+
+Recommend top-k movies that a user have not ever seen.
+```sql
+set hivevar:userid=1;
+set hivevar:topk=5;
+
+select
+  t1.movieid, 
+  mf_predict(t2.Pu, t1.Qi, t2.Bu, t1.Bi, ${mu}) as predicted
+from (
+  select
+    idx movieid,
+    Qi, 
+    Bi
+  from
+    sgd_model p
+  where
+    p.idx NOT IN 
+      (select movieid from training where userid=${userid})
+) t1 CROSS JOIN (
+  select
+    Pu,
+    Bu
+  from 
+    sgd_model
+  where
+    idx = ${userid}
+) t2
+order by
+  predicted DESC
+limit ${topk};
+```
+
+| movieid | predicted |
+|--------:|----------:|
+| 318     | 4.8051853 |
+| 2503    | 4.788541  |
+| 53      | 4.7518783 |
+| 904     | 4.7463417 |
+| 953     | 4.732769  |
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/370e2aa3/docs/gitbook/recommend/news20.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/recommend/news20.md b/docs/gitbook/recommend/news20.md
new file mode 100644
index 0000000..e69de29

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/370e2aa3/docs/gitbook/recommend/news20_bbit_minhash.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/recommend/news20_bbit_minhash.md b/docs/gitbook/recommend/news20_bbit_minhash.md
new file mode 100644
index 0000000..72ba7f2
--- /dev/null
+++ b/docs/gitbook/recommend/news20_bbit_minhash.md
@@ -0,0 +1,50 @@
+# Function Signature of bbit_minhash
+
+```
+Text bbit_minhash(array<int|string> features)
+Text bbit_minhash(array<int|string> features, int numHashes=128)
+Text bbit_minhash(array<int|string> features, boolean discardWeight=false)
+Text bbit_minhash(array<int|string> features, int numHashes=128, boolean discardWeight=false)
+```
+
+# Create a signature for each article
+
+```sql
+create table new20mc_with_signature
+as
+select
+  rowid, 
+  bbit_minhash(features, false) as signature
+from
+  news20mc_train;
+```
+
+# kNN brute-force search using b-Bit minhash
+```sql
+set hivevar:topn=10;
+
+select
+  t1.rowid, 
+  jaccard_similarity(t1.signature, q1.signature,128) as similarity
+--  , popcnt(t1.signature, q1.signature) as popcnt
+from
+  new20mc_with_signature t1 
+  CROSS JOIN 
+  (select bbit_minhash(features,128,false) as signature from news20mc_test where rowid = 1) q1
+order by
+  similarity DESC
+limit ${topn};
+```
+
+|rowid  | similarity | popcnt |
+|:------|------------|-------:|
+| 11952 | 0.390625   | 41 |
+| 10748 | 0.359375   | 41 |
+| 12902 | 0.34375    | 45 |
+| 3087  | 0.328125   | 48 |
+| 3     | 0.328125   | 37 |
+| 11493 | 0.328125   | 38 |
+| 3839  | 0.328125   | 41 |
+| 12669 | 0.328125   | 37 |
+| 13604 | 0.3125     | 41 |
+| 6333  | 0.3125     | 39 |
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/370e2aa3/docs/gitbook/recommend/news20_jaccard.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/recommend/news20_jaccard.md b/docs/gitbook/recommend/news20_jaccard.md
new file mode 100644
index 0000000..ea6f8cc
--- /dev/null
+++ b/docs/gitbook/recommend/news20_jaccard.md
@@ -0,0 +1,123 @@
+List related (similar) articles for each article.
+
+# Preparation
+```sql
+use news20;
+
+delete jar /home/myui/tmp/hivemall.jar;
+add jar /home/myui/tmp/hivemall.jar;
+
+source /home/myui/tmp/define-all.hive;
+```
+
+# Extract clusters
+```sql
+set hivevar:hashes=100; -- Generate N sets of minhash values for each row (DEFAULT: 5)
+set hivevar:keygroups=2; -- Use K minhash value for generating a resulting value (DEFAULT: 2)
+
+create table news20_clusterid_assign
+as
+select 
+  -- minhash(rowid, features) as (clusterId, rowid)
+  minhash(rowid, features, "-n ${hashes} -k ${keygroups}") as (clusterId, rowid)
+from 
+  news20mc_train;
+
+--set hivevar:min_cluster_size=5;
+
+create or replace view news20_cluster
+as
+select
+  clusterId, 
+  collect_set(rowid) as rowids
+from 
+  news20_clusterid_assign
+group by clusterId
+-- having size(rowids) > ${min_cluster_size}
+;
+```
+
+# Get recommendations
+
+```sql
+create table news20_similar_articles
+as
+WITH t1 as (
+select
+  l.rowid,
+  r.rowid as other_id,
+  count(1) as cnt
+from
+  news20_clusterid_assign l 
+  LEFT OUTER JOIN
+  news20_clusterid_assign r
+     ON (l.clusterid = r.clusterid)
+where 
+  l.rowid != r.rowid
+group by
+  l.rowid, r.rowid
+having 
+-- 10/${hashes}=10/100=0.1 (filter by a pseudo Jaccard similarity by Minhash is greater than or equals to 0.1)
+  cnt >= 10 
+)
+select
+  rowid,
+  collect_set(other_id) as related_articles
+from 
+  t1
+group by
+  rowid
+-- order by rowid asc
+;
+```
+
+List all possible clusters w/o using a similarity threshold:
+```sql
+create table news20_similar_articles2
+as
+select
+  l.rowid,
+  collect_set(r.rowid) as related_articles
+from
+  news20_clusterid_assign l 
+  LEFT OUTER JOIN
+  news20_clusterid_assign r
+     ON (l.clusterid = r.clusterid)
+where 
+  l.rowid != r.rowid
+group by
+  l.rowid
+-- order by rowid asc
+;
+```
+
+## Jaccard similarity computation using k-Minhash
+
+```sql
+create table news20_jaccard_similarity
+as
+WITH t1 as (
+select
+  l.rowid,
+  r.rowid as other_id,
+  count(1) / ${hashes} as similarity
+from
+  news20_clusterid_assign l 
+  JOIN news20_clusterid_assign r
+     ON (l.clusterid = r.clusterid)
+where 
+  l.rowid != r.rowid
+group by
+  l.rowid, r.rowid
+)
+select
+  rowid,
+  other_id,
+  similarity,
+  1.0 - similarity as distance
+from
+  t1
+where
+  similarity >= 0.1
+;
+```
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/370e2aa3/docs/gitbook/recommend/news20_knn.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/recommend/news20_knn.md b/docs/gitbook/recommend/news20_knn.md
new file mode 100644
index 0000000..0471a2e
--- /dev/null
+++ b/docs/gitbook/recommend/news20_knn.md
@@ -0,0 +1,103 @@
+# Extract clusters and assign N cluster IDs to each article
+```
+create or replace view news20_cluster
+as
+select 
+  minhash(rowid, features) as (clusterId, rowid)
+from 
+  news20mc_train;
+
+create table news20_with_clusterid
+as
+select 
+  t1.clusterid, 
+  t1.rowid, 
+  o1.features
+from 
+  news20_cluster t1
+  JOIN news20mc_train o1 ON (t1.rowid = o1.rowid);
+```
+
+# Query expression with cluster id
+```
+set hivevar:noWeight=false;
+
+create table extract_target_cluster
+as
+select 
+  features,
+  clusterid
+from (
+  select 
+     features,
+     minhashes(features,${noWeight}) as clusters
+  from 
+     news20mc_test 
+  where 
+     rowid = 1
+) t1
+LATERAL VIEW explode(clusters) t2 AS clusterid;
+```
+
+# kNN search using minhashing
+```sql
+set hivevar:topn=10;
+
+select 
+  t1.rowid, 
+  cosine_similarity(t1.features, q1.features, false) as similarity
+from
+  news20_with_clusterid t1
+  JOIN extract_target_cluster q1 ON (t1.clusterid = q1.clusterid)
+order by
+  similarity DESC
+limit ${topn};
+```
+
+> Time taken: 22.161 seconds
+
+|rowid  | similarity |
+|:------|-----------:|
+| 2182  | 0.21697778  |
+| 5622  | 0.21483186  |
+| 962   |  0.13240485 |
+| 12242 |  0.12158953 |
+| 5102  | 0.11168713  |
+| 8562  | 0.107470974 |
+| 14396 |0.09949879   |
+| 2542  | 0.09011547  |
+| 1645  | 0.08894014  |
+| 2862  | 0.08800333  |
+
+# Brute force kNN search (based on cosine similarity)
+```sql
+select
+  t1.rowid,
+  cosine_similarity(t1.features, q1.features) as similarity -- hive v0.3.2 or later
+  -- cosine_similarity(t1.features, q1.features, false) as similarity -- hive v0.3.1 or before
+from 
+  news20mc_train t1
+  CROSS JOIN
+  (select features from news20mc_test where rowid = 1) q1
+ORDER BY
+  similarity DESC
+limit ${topn};
+```
+
+> Time taken: 24.335 seconds
+
+|rowid  | similarity |
+|:------|-----------:|
+| 12902 | 0.47759432 |
+| 7922  | 0.4184913  |
+| 2382  | 0.21919869 |
+| 2182  | 0.21697778 |
+| 5622  | 0.21483186 |
+| 9562  | 0.21223815 |
+| 3222  | 0.164399   |
+| 11202 | 0.16439897 |
+| 10122 | 0.1620197  |
+| 8482  | 0.15229382 |
+
+
+Refer [this page](https://github.com/myui/hivemall/wiki/Efficient-Top-k-computation-on-Apache-Hive-using-Hivemall-UDTF#top-k-similarity-computation) for efficient top-k kNN computation.
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/370e2aa3/docs/gitbook/regression/e2006.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/regression/e2006.md b/docs/gitbook/regression/e2006.md
new file mode 100644
index 0000000..e69de29

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/370e2aa3/docs/gitbook/regression/e2006_arow.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/regression/e2006_arow.md b/docs/gitbook/regression/e2006_arow.md
new file mode 100644
index 0000000..a3b60eb
--- /dev/null
+++ b/docs/gitbook/regression/e2006_arow.md
@@ -0,0 +1,259 @@
+http://www.csie.ntu.edu.tw/~cjlin/libsvmtools/datasets/regression.html#E2006-tfidf
+
+---
+#[PA1a]
+
+##Training
+```sql
+set mapred.reduce.tasks=64;
+drop table e2006tfidf_pa1a_model ;
+create table e2006tfidf_pa1a_model as
+select 
+ feature,
+ avg(weight) as weight
+from 
+ (select 
+     train_pa1a_regr(addBias(features),target) as (feature,weight)
+  from 
+     e2006tfidf_train_x3
+ ) t 
+group by feature;
+set mapred.reduce.tasks=-1;
+```
+_Caution: Do not use voted_avg() for regression. voted_avg() is for classification._
+
+## prediction
+```sql
+create or replace view e2006tfidf_pa1a_predict
+as
+select
+  t.rowid, 
+  sum(m.weight * t.value) as predicted
+from 
+  e2006tfidf_test_exploded t LEFT OUTER JOIN
+  e2006tfidf_pa1a_model m ON (t.feature = m.feature)
+group by
+  t.rowid;
+```
+
+## evaluation
+```sql
+drop table e2006tfidf_pa1a_submit;
+create table e2006tfidf_pa1a_submit as
+select 
+  t.target as actual, 
+  p.predicted as predicted
+from 
+  e2006tfidf_test t JOIN e2006tfidf_pa1a_predict p 
+    on (t.rowid = p.rowid);
+
+select avg(actual), avg(predicted) from e2006tfidf_pa1a_submit;
+```
+> -3.8200363760415414     -3.8869923258589476
+
+```sql
+set hivevar:mean_actual=-3.8200363760415414;
+
+select 
+   sqrt(sum(pow(predicted - actual,2.0))/count(1)) as RMSE, 
+   sum(pow(predicted - actual,2.0))/count(1) as MSE, 
+   sum(abs(predicted - actual))/count(1) as MAE,
+   1 - sum(pow(actual - predicted,2.0)) / sum(pow(actual - ${mean_actual},2.0)) as R2
+from 
+   e2006tfidf_pa1a_submit;
+```
+> 0.3797959864675519      0.14424499133686086     0.23846059576113587     0.5010367946980386
+
+---
+#[PA2a]
+
+##Training
+```sql
+set mapred.reduce.tasks=64;
+drop table e2006tfidf_pa2a_model;
+create table e2006tfidf_pa2a_model as
+select 
+ feature,
+ avg(weight) as weight
+from 
+ (select 
+     train_pa2a_regr(addBias(features),target) as (feature,weight)
+  from 
+     e2006tfidf_train_x3
+ ) t 
+group by feature;
+set mapred.reduce.tasks=-1;
+```
+
+## prediction
+```sql
+create or replace view e2006tfidf_pa2a_predict
+as
+select
+  t.rowid, 
+  sum(m.weight * t.value) as predicted
+from 
+  e2006tfidf_test_exploded t LEFT OUTER JOIN
+  e2006tfidf_pa2a_model m ON (t.feature = m.feature)
+group by
+  t.rowid;
+```
+
+## evaluation
+```sql
+drop table e2006tfidf_pa2a_submit;
+create table e2006tfidf_pa2a_submit as
+select 
+  t.target as actual, 
+  pd.predicted as predicted
+from 
+  e2006tfidf_test t JOIN e2006tfidf_pa2a_predict pd 
+    on (t.rowid = pd.rowid);
+
+select avg(actual), avg(predicted) from e2006tfidf_pa2a_submit;
+```
+> -3.8200363760415414     -3.9124877451612488
+
+```sql
+set hivevar:mean_actual=-3.8200363760415414;
+
+select 
+   sqrt(sum(pow(predicted - actual,2.0))/count(1)) as RMSE, 
+   sum(pow(predicted - actual,2.0))/count(1) as MSE, 
+   sum(abs(predicted - actual))/count(1) as MAE,
+   1 - sum(pow(actual - predicted,2.0)) / sum(pow(actual - ${mean_actual},2.0)) as R2
+from 
+   e2006tfidf_pa2a_submit;
+```
+> 0.38538660838804495     0.14852283792484033     0.2466732002711477      0.48623913673053565
+
+---
+#[AROW]
+
+##Training
+```sql
+set mapred.reduce.tasks=64;
+drop table e2006tfidf_arow_model ;
+create table e2006tfidf_arow_model as
+select 
+ feature,
+ -- avg(weight) as weight -- [hivemall v0.1]
+ argmin_kld(weight, covar) as weight -- [hivemall v0.2 or later]
+from 
+ (select 
+     -- train_arow_regr(addBias(features),target) as (feature,weight)    -- [hivemall v0.1]
+     train_arow_regr(addBias(features),target) as (feature,weight,covar) -- [hivemall v0.2 or later]
+  from 
+     e2006tfidf_train_x3
+ ) t 
+group by feature;
+set mapred.reduce.tasks=-1;
+```
+
+## prediction
+```sql
+create or replace view e2006tfidf_arow_predict
+as
+select
+  t.rowid, 
+  sum(m.weight * t.value) as predicted
+from 
+  e2006tfidf_test_exploded t LEFT OUTER JOIN
+  e2006tfidf_arow_model m ON (t.feature = m.feature)
+group by
+  t.rowid;
+```
+
+## evaluation
+```sql
+drop table e2006tfidf_arow_submit;
+create table e2006tfidf_arow_submit as
+select 
+  t.target as actual, 
+  p.predicted as predicted
+from 
+  e2006tfidf_test t JOIN e2006tfidf_arow_predict p
+    on (t.rowid = p.rowid);
+
+select avg(actual), avg(predicted) from e2006tfidf_arow_submit;
+```
+> -3.8200363760415414     -3.8692518911517433
+
+```sql
+set hivevar:mean_actual=-3.8200363760415414;
+
+select 
+   sqrt(sum(pow(predicted - actual,2.0))/count(1)) as RMSE, 
+   sum(pow(predicted - actual,2.0))/count(1) as MSE, 
+   sum(abs(predicted - actual))/count(1) as MAE,
+   1 - sum(pow(actual - predicted,2.0)) / sum(pow(actual - ${mean_actual},2.0)) as R2
+from 
+   e2006tfidf_arow_submit;
+```
+> 0.37862513029019407     0.14335698928726642     0.2368787001269389      0.5041085155590119
+
+--- 
+#[AROWe]
+AROWe is a modified version of AROW that uses Hinge loss (epsilion = 0.1)
+
+##Training
+```sql
+set mapred.reduce.tasks=64;
+drop table e2006tfidf_arowe_model ;
+create table e2006tfidf_arowe_model as
+select 
+ feature,
+ -- avg(weight) as weight -- [hivemall v0.1]
+ argmin_kld(weight, covar) as weight -- [hivemall v0.2 or later]
+from 
+ (select 
+     -- train_arowe_regr(addBias(features),target) as (feature,weight)    -- [hivemall v0.1]
+     train_arowe_regr(addBias(features),target) as (feature,weight,covar) -- [hivemall v0.2 or later]
+  from 
+     e2006tfidf_train_x3
+ ) t 
+group by feature;
+set mapred.reduce.tasks=-1;
+```
+
+## prediction
+```sql
+create or replace view e2006tfidf_arowe_predict
+as
+select
+  t.rowid, 
+  sum(m.weight * t.value) as predicted
+from 
+  e2006tfidf_test_exploded t LEFT OUTER JOIN
+  e2006tfidf_arowe_model m ON (t.feature = m.feature)
+group by
+  t.rowid;
+```
+
+## evaluation
+```sql
+drop table e2006tfidf_arowe_submit;
+create table e2006tfidf_arowe_submit as
+select 
+  t.target as actual, 
+  p.predicted as predicted
+from 
+  e2006tfidf_test t JOIN e2006tfidf_arowe_predict p
+    on (t.rowid = p.rowid);
+
+select avg(actual), avg(predicted) from e2006tfidf_arowe_submit;
+```
+> -3.8200363760415414     -3.86494905688414
+
+```sql
+set hivevar:mean_actual=-3.8200363760415414;
+
+select 
+   sqrt(sum(pow(predicted - actual,2.0))/count(1)) as RMSE, 
+   sum(pow(predicted - actual,2.0))/count(1) as MSE, 
+   sum(abs(predicted - actual))/count(1) as MAE,
+   1 - sum(pow(actual - predicted,2.0)) / sum(pow(actual - ${mean_actual},2.0)) as R2
+from 
+   e2006tfidf_arowe_submit;
+```
+> 0.37789148212861856     0.14280197226536404     0.2357339155291536      0.5060283955470721
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/370e2aa3/docs/gitbook/regression/e2006_dataset.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/regression/e2006_dataset.md b/docs/gitbook/regression/e2006_dataset.md
new file mode 100644
index 0000000..329fb39
--- /dev/null
+++ b/docs/gitbook/regression/e2006_dataset.md
@@ -0,0 +1,72 @@
+http://www.csie.ntu.edu.tw/~cjlin/libsvmtools/datasets/regression.html#E2006-tfidf
+
+Prerequisite
+============
+* [hivemall.jar](https://github.com/myui/hivemall/tree/master/target/hivemall.jar)
+* [conv.awk](https://github.com/myui/hivemall/tree/master/scripts/misc/conv.awk)
+* [define-all.hive](https://github.com/myui/hivemall/tree/master/scripts/ddl/define-all.hive)
+
+Data preparation
+================
+
+```sh
+cd /mnt/archive/datasets/regression/E2006-tfidf
+awk -f conv.awk E2006.train > E2006.train.tsv
+awk -f conv.awk  E2006.test > E2006.test.tsv
+
+hadoop fs -mkdir -p /dataset/E2006-tfidf/train
+hadoop fs -mkdir -p /dataset/E2006-tfidf/test
+hadoop fs -put E2006.train.tsv /dataset/E2006-tfidf/train
+hadoop fs -put E2006.test.tsv /dataset/E2006-tfidf/test
+```
+
+```sql
+create database E2006;
+use E2006;
+
+delete jar /home/myui/tmp/hivemall.jar;
+add jar /home/myui/tmp/hivemall.jar;
+
+source /home/myui/tmp/define-all.hive;
+
+Create external table e2006tfidf_train (
+  rowid int,
+  target float,
+  features ARRAY<STRING>
+) 
+ROW FORMAT DELIMITED FIELDS TERMINATED BY '\t' COLLECTION ITEMS TERMINATED BY "," 
+STORED AS TEXTFILE LOCATION '/dataset/E2006-tfidf/train';
+
+Create external table e2006tfidf_test (
+  rowid int, 
+  target float,
+  features ARRAY<STRING>
+) 
+ROW FORMAT DELIMITED FIELDS TERMINATED BY '\t' COLLECTION ITEMS TERMINATED BY "," 
+STORED AS TEXTFILE LOCATION '/dataset/E2006-tfidf/test';
+
+create table e2006tfidf_test_exploded as
+select 
+  rowid,
+  target,
+  split(feature,":")[0] as feature,
+  cast(split(feature,":")[1] as float) as value
+  -- hivemall v0.3.1 or later
+  -- extract_feature(feature) as feature,
+  -- extract_weight(feature) as value
+from 
+  e2006tfidf_test LATERAL VIEW explode(addBias(features)) t AS feature;
+```
+
+## Amplify training examples (global shuffle)
+```sql
+-- set mapred.reduce.tasks=32;
+set hivevar:seed=31;
+set hivevar:xtimes=3;
+create or replace view e2006tfidf_train_x3 as 
+select * from (
+select amplify(${xtimes}, *) as (rowid, target, features) from e2006tfidf_train
+) t
+CLUSTER BY rand(${seed});
+-- set mapred.reduce.tasks=-1;
+```
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/370e2aa3/docs/gitbook/regression/kddcup12tr2.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/regression/kddcup12tr2.md b/docs/gitbook/regression/kddcup12tr2.md
new file mode 100644
index 0000000..e69de29

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/370e2aa3/docs/gitbook/regression/kddcup12tr2_adagrad.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/regression/kddcup12tr2_adagrad.md b/docs/gitbook/regression/kddcup12tr2_adagrad.md
new file mode 100644
index 0000000..e6c8eb4
--- /dev/null
+++ b/docs/gitbook/regression/kddcup12tr2_adagrad.md
@@ -0,0 +1,109 @@
+_Note adagrad/adadelta is supported from hivemall v0.3b2 or later (or in the master branch)._
+
+# Preparation 
+```sql
+add jar ./tmp/hivemall-with-dependencies.jar;
+source ./tmp/define-all.hive;
+
+use kdd12track2;
+
+-- SET mapreduce.framework.name=yarn;
+-- SET hive.execution.engine=mr;
+-- SET mapreduce.framework.name=yarn-tez;
+-- SET hive.execution.engine=tez;
+SET mapred.reduce.tasks=32; -- [optional] set the explicit number of reducers to make group-by aggregation faster
+```
+
+# AdaGrad
+```sql
+drop table adagrad_model;
+create table adagrad_model 
+as
+select 
+ feature,
+ avg(weight) as weight
+from 
+ (select 
+     adagrad(features,label) as (feature,weight)
+  from 
+     training_orcfile
+ ) t 
+group by feature;
+
+drop table adagrad_predict;
+create table adagrad_predict
+  ROW FORMAT DELIMITED 
+    FIELDS TERMINATED BY "\t"
+    LINES TERMINATED BY "\n"
+  STORED AS TEXTFILE
+as
+select
+  t.rowid, 
+  sigmoid(sum(m.weight)) as prob
+from 
+  testing_exploded  t LEFT OUTER JOIN
+  adagrad_model m ON (t.feature = m.feature)
+group by 
+  t.rowid
+order by 
+  rowid ASC;
+```
+
+```sh
+hadoop fs -getmerge /user/hive/warehouse/kdd12track2.db/adagrad_predict adagrad_predict.tbl
+
+gawk -F "\t" '{print $2;}' adagrad_predict.tbl > adagrad_predict.submit
+
+pypy scoreKDD.py KDD_Track2_solution.csv adagrad_predict.submit
+```
+>AUC(SGD) : 0.739351
+
+>AUC(ADAGRAD) : 0.743279
+
+# AdaDelta
+```sql
+drop table adadelta_model;
+create table adadelta_model 
+as
+select 
+ feature,
+ cast(avg(weight) as float) as weight
+from 
+ (select 
+     adadelta(features,label) as (feature,weight)
+  from 
+     training_orcfile
+ ) t 
+group by feature;
+
+drop table adadelta_predict;
+create table adadelta_predict
+  ROW FORMAT DELIMITED 
+    FIELDS TERMINATED BY "\t"
+    LINES TERMINATED BY "\n"
+  STORED AS TEXTFILE
+as
+select
+  t.rowid, 
+  sigmoid(sum(m.weight)) as prob
+from 
+  testing_exploded  t LEFT OUTER JOIN
+  adadelta_model m ON (t.feature = m.feature)
+group by 
+  t.rowid
+order by 
+  rowid ASC;
+```
+
+```sh
+hadoop fs -getmerge /user/hive/warehouse/kdd12track2.db/adadelta_predict adadelta_predict.tbl
+
+gawk -F "\t" '{print $2;}' adadelta_predict.tbl > adadelta_predict.submit
+
+pypy scoreKDD.py KDD_Track2_solution.csv adadelta_predict.submit
+```
+>AUC(SGD) : 0.739351
+
+>AUC(ADAGRAD) : 0.743279
+
+> AUC(AdaDelta) : 0.746878
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/370e2aa3/docs/gitbook/regression/kddcup12tr2_dataset.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/regression/kddcup12tr2_dataset.md b/docs/gitbook/regression/kddcup12tr2_dataset.md
new file mode 100644
index 0000000..8713e99
--- /dev/null
+++ b/docs/gitbook/regression/kddcup12tr2_dataset.md
@@ -0,0 +1,227 @@
+The task is predicting the click through rate (CTR) of advertisement, meaning that we are to predict the probability of each ad being clicked. 
+http://www.kddcup2012.org/c/kddcup2012-track2
+
+---
+
+**Dataset**  
+
+|    File    |    Size     |    Records   |
+|:-----------|------------:|:------------:|
+| KDD_Track2_solution.csv | 244MB | 20,297,595 (20,297,594 w/o header) |
+| descriptionid_tokensid.txt | 268MB | 3,171,830 |
+| purchasedkeywordid_tokensid.txt | 26MB | 1,249,785 |
+| queryid_tokensid.txt | 704MB | 26,243,606 |
+| test.txt | 1.3GB | 20,297,594 |
+| titleid_tokensid.txt | 171MB | 4,051,441 |
+| training.txt | 9.9GB | 149,639,105 |
+| serid_profile.txt | 283MB | 23,669,283 |
+
+![tables](https://raw.github.com/myui/hivemall/master/resources/examples/kddtrack2/tables.png)
+
+_Tokens are actually not used in this example. Try using them on your own._
+
+---
+
+```sql
+create database kdd12track2;
+use kdd12track2;
+
+delete jar /tmp/hivemall.jar;
+add jar /tmp/hivemall.jar;
+source /tmp/define-all.hive;
+
+Create external table training (
+  RowID BIGINT,
+  Clicks INT, 
+  Impression INT, 
+  DisplayURL STRING, 
+  AdID INT,
+  AdvertiserID INT, 
+  Depth SMALLINT, 
+  Position SMALLINT, 
+  QueryID INT, 
+  KeywordID INT,
+  TitleID INT, 
+  DescriptionID INT, 
+  UserID INT
+) ROW FORMAT DELIMITED FIELDS TERMINATED BY '\t' STORED AS TEXTFILE LOCATION '/kddcup2012/track2/training';
+
+Create external table testing (
+  RowID BIGINT,
+  DisplayURL STRING, 
+  AdID INT,
+  AdvertiserID INT, 
+  Depth SMALLINT, 
+  Position SMALLINT, 
+  QueryID INT, 
+  KeywordID INT,
+  TitleID INT, 
+  DescriptionID INT, 
+  UserID INT
+) ROW FORMAT DELIMITED FIELDS TERMINATED BY '\t' STORED AS TEXTFILE LOCATION '/kddcup2012/track2/testing';
+
+Create external table user (
+  UserID INT, 
+  Gender TINYINT,
+  Age TINYINT
+) ROW FORMAT DELIMITED FIELDS TERMINATED BY '\t' STORED AS TEXTFILE LOCATION '/kddcup2012/track2/user';
+
+Create external table query (
+  QueryID INT,
+  Tokens STRING
+) ROW FORMAT DELIMITED FIELDS TERMINATED BY '\t' STORED AS TEXTFILE LOCATION '/kddcup2012/track2/query';
+
+Create external table keyword (
+  KeywordID INT,
+  Tokens STRING
+) ROW FORMAT DELIMITED FIELDS TERMINATED BY '\t' STORED AS TEXTFILE LOCATION '/kddcup2012/track2/keyword';
+
+Create external table title (
+  TitleID INT, 
+  Tokens STRING
+) ROW FORMAT DELIMITED FIELDS TERMINATED BY '\t' STORED AS TEXTFILE LOCATION '/kddcup2012/track2/title';
+
+Create external table description (
+  DescriptionID INT,
+  Tokens STRING
+) ROW FORMAT DELIMITED FIELDS TERMINATED BY '\t' STORED AS TEXTFILE LOCATION '/kddcup2012/track2/description';
+
+Create external table solution (
+   RowID BIGINT,
+   Clicks INT,
+   Impressions INT,
+   Private BOOLEAN 
+) ROW FORMAT DELIMITED FIELDS TERMINATED BY ',' STORED AS TEXTFILE LOCATION '/kddcup2012/track2/solution';
+```
+
+```sh
+gawk '{print NR"\t"$0;}' training.txt | \
+hadoop fs -put - /kddcup2012/track2/training/training.tsv
+
+gawk '{print NR"\t"$0;}' test.txt | \
+hadoop fs -put - /kddcup2012/track2/testing/test.tsv
+
+hadoop fs -put userid_profile.txt /kddcup2012/track2/user/user.tsv
+
+tail -n +2 KDD_Track2_solution.csv | sed -e 's/Public/FALSE/g' | sed -e 's/Private/TRUE/g' | gawk '{print NR","$0;}' \
+hadoop fs -put - /kddcup2012/track2/solution/solution.csv
+
+hadoop fs -put  queryid_tokensid.txt /kddcup2012/track2/query/tokensid.tsv
+hadoop fs -put purchasedkeywordid_tokensid.txt /kddcup2012/track2/keyword/tokensid.tsv
+hadoop fs -put titleid_tokensid.txt /kddcup2012/track2/title/tokensid.tsv
+hadoop fs -put descriptionid_tokensid.txt /kddcup2012/track2/description/tokensid.tsv
+```
+
+## Converting feature representation by feature hashing
+http://en.wikipedia.org/wiki/Feature_hashing
+
+*mhash* is the MurmurHash3 function to convert a feature vector into a hash value.
+
+```sql
+create or replace view training2 as
+select
+  rowid,
+  clicks,
+  (impression - clicks) as noclick,
+  mhash(concat("1:", displayurl)) as displayurl, 
+  mhash(concat("2:", adid)) as adid, 
+  mhash(concat("3:", advertiserid)) as advertiserid, 
+  mhash(concat("4:", depth)) as depth, 
+  mhash(concat("5:", position)) as position, 
+  mhash(concat("6:", queryid)) as queryid, 
+  mhash(concat("7:", keywordid)) as keywordid, 
+  mhash(concat("8:", titleid)) as titleid, 
+  mhash(concat("9:", descriptionid)) as descriptionid, 
+  mhash(concat("10:", userid)) as userid, 
+  mhash(concat("11:", COALESCE(gender,"0"))) as gender, 
+  mhash(concat("12:", COALESCE(age,"-1"))) as age, 
+  -1 as bias
+from (
+select
+  t.*,
+  u.gender,
+  u.age
+from 
+  training t 
+  LEFT OUTER JOIN user u 
+    on t.userid = u.userid
+) t;
+
+create or replace view testing2 as
+select
+  rowid, 
+  array(displayurl, adid, advertiserid, depth, position, queryid, keywordid, titleid, descriptionid, userid, gender, age, bias) 
+    as features
+from (
+select
+  rowid,
+  mhash(concat("1:", displayurl)) as displayurl, 
+  mhash(concat("2:", adid)) as adid, 
+  mhash(concat("3:", advertiserid)) as advertiserid, 
+  mhash(concat("4:", depth)) as depth, 
+  mhash(concat("5:", position)) as position, 
+  mhash(concat("6:", queryid)) as queryid, 
+  mhash(concat("7:", keywordid)) as keywordid, 
+  mhash(concat("8:", titleid)) as titleid, 
+  mhash(concat("9:", descriptionid)) as descriptionid, 
+  mhash(concat("10:", userid)) as userid, 
+  mhash(concat("11:", COALESCE(gender,"0"))) as gender, 
+  mhash(concat("12:", COALESCE(age,"-1"))) as age, 
+  -1 as bias
+from (
+select
+  t.*,
+  u.gender,
+  u.age
+from 
+  testing t 
+  LEFT OUTER JOIN user u 
+    on t.userid = u.userid
+) t1
+) t2;
+```
+
+## Compressing large training tables
+```sql
+create table training_orcfile (
+ rowid bigint,
+ label float,
+ features array<int>
+) STORED AS orc tblproperties ("orc.compress"="SNAPPY");
+```
+_Caution: Joining between training table and user table takes a long time. Consider not to use gender and age and avoid joins if your Hadoop cluster is small._
+
+[kddconv.awk](https://github.com/myui/hivemall/blob/master/resources/examples/kddtrack2/kddconv.awk)
+
+```sql
+add file /tmp/kddconv.awk;
+
+-- SET mapred.reduce.tasks=64;
+-- SET hive.auto.convert.join=false;
+
+INSERT OVERWRITE TABLE training_orcfile 
+select transform(*) 
+  ROW FORMAT DELIMITED
+     FIELDS TERMINATED BY "\t"
+     LINES TERMINATED BY "\n"
+using 'gawk -f kddconv.awk' 
+  as (rowid BIGINT, label FLOAT, features ARRAY<INT>)
+  ROW FORMAT DELIMITED
+     FIELDS TERMINATED BY "\t"
+     COLLECTION ITEMS TERMINATED BY ","
+     LINES TERMINATED BY "\n"
+from training2
+CLUSTER BY rand();
+
+-- SET mapred.reduce.tasks=-1;
+-- SET hive.auto.convert.join=true;
+
+create table testing_exploded as
+select 
+  rowid,
+  feature
+from 
+  testing2 
+  LATERAL VIEW explode(features) t AS feature;
+```
+_Caution: We recommend you to set "mapred.reduce.tasks" in the above example to partition the training_orcfile table into pieces._
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/370e2aa3/docs/gitbook/regression/kddcup12tr2_lr.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/regression/kddcup12tr2_lr.md b/docs/gitbook/regression/kddcup12tr2_lr.md
new file mode 100644
index 0000000..0ff9b97
--- /dev/null
+++ b/docs/gitbook/regression/kddcup12tr2_lr.md
@@ -0,0 +1,141 @@
+The task is predicting the click through rate (CTR) of advertisement, meaning that we are to predict the probability of each ad being clicked.   
+http://www.kddcup2012.org/c/kddcup2012-track2
+
+_Caution: This example just shows a baseline result. Use token tables and amplifier to get better AUC score._
+
+---
+Logistic Regression
+===============
+
+## Training
+```sql
+use kdd12track2;
+
+-- set mapred.max.split.size=134217728; -- [optional] set if OOM caused at mappers on training
+-- SET mapred.max.split.size=67108864;
+select count(1) from training_orcfile;
+```
+> 235582879
+
+235582879 / 56 (mappers) = 4206837
+
+```sql
+set hivevar:total_steps=5000000;
+-- set mapred.reduce.tasks=64; -- [optional] set the explicit number of reducers to make group-by aggregation faster
+
+drop table lr_model;
+create table lr_model 
+as
+select 
+ feature,
+ cast(avg(weight) as float) as weight
+from 
+ (select 
+     logress(features, label, "-total_steps ${total_steps}") as (feature,weight)
+     -- logress(features, label) as (feature,weight)
+  from 
+     training_orcfile
+ ) t 
+group by feature;
+
+-- set mapred.max.split.size=-1; -- reset to the default value
+```
+_Note: Setting the "-total_steps" option is optional._
+
+## Prediction
+```
+drop table lr_predict;
+create table lr_predict
+  ROW FORMAT DELIMITED 
+    FIELDS TERMINATED BY "\t"
+    LINES TERMINATED BY "\n"
+  STORED AS TEXTFILE
+as
+select
+  t.rowid, 
+  sigmoid(sum(m.weight)) as prob
+from 
+  testing_exploded  t LEFT OUTER JOIN
+  lr_model m ON (t.feature = m.feature)
+group by 
+  t.rowid
+order by 
+  rowid ASC;
+```
+## Evaluation
+
+[scoreKDD.py](https://github.com/myui/hivemall/blob/master/resources/examples/kddtrack2/scoreKDD.py)
+
+```sh
+hadoop fs -getmerge /user/hive/warehouse/kdd12track2.db/lr_predict lr_predict.tbl
+
+gawk -F "\t" '{print $2;}' lr_predict.tbl > lr_predict.submit
+
+pypy scoreKDD.py KDD_Track2_solution.csv  lr_predict.submit
+```
+_Note: You can use python instead of pypy._
+
+| Measure | Score |
+|:-----------|------------:|
+| AUC  | 0.741111 |
+| NWMAE | 0.045493 |
+| WRMSE | 0.142395 |
+---
+Passive Aggressive
+===============
+
+## Training
+```
+drop table pa_model;
+create table pa_model 
+as
+select 
+ feature,
+ cast(avg(weight) as float) as weight
+from 
+ (select 
+     train_pa1a_regr(features,label) as (feature,weight)
+  from 
+     training_orcfile
+ ) t 
+group by feature;
+```
+_PA1a is recommended when using PA for regression._
+
+## Prediction
+```
+drop table pa_predict;
+create table pa_predict
+  ROW FORMAT DELIMITED 
+    FIELDS TERMINATED BY "\t"
+    LINES TERMINATED BY "\n"
+  STORED AS TEXTFILE
+as
+select
+  t.rowid, 
+  sum(m.weight) as prob
+from 
+  testing_exploded  t LEFT OUTER JOIN
+  pa_model m ON (t.feature = m.feature)
+group by 
+  t.rowid
+order by 
+  rowid ASC;
+```
+_The "prob" of PA can be used only for ranking and can have a negative value. A higher weight means much likely to be clicked. Note that AUC is sort a measure for evaluating ranking accuracy._
+
+## Evaluation
+
+```sh
+hadoop fs -getmerge /user/hive/warehouse/kdd12track2.db/pa_predict pa_predict.tbl
+
+gawk -F "\t" '{print $2;}' pa_predict.tbl > pa_predict.submit
+
+pypy scoreKDD.py KDD_Track2_solution.csv  pa_predict.submit
+```
+
+| Measure | Score |
+|:-----------|------------:|
+| AUC  | 0.739722 |
+| NWMAE | 0.049582 |
+| WRMSE | 0.143698 |
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/370e2aa3/docs/gitbook/regression/kddcup12tr2_lr_amplify.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/regression/kddcup12tr2_lr_amplify.md b/docs/gitbook/regression/kddcup12tr2_lr_amplify.md
new file mode 100644
index 0000000..4df124e
--- /dev/null
+++ b/docs/gitbook/regression/kddcup12tr2_lr_amplify.md
@@ -0,0 +1,103 @@
+This article explains *amplify* technique that is useful for improving prediction score.
+
+Iterations are mandatory in machine learning (e.g., in [stochastic gradient descent](http://en.wikipedia.org/wiki/Stochastic_gradient_descent)) to get good prediction models. However, MapReduce is known to be not suited for iterative algorithms because IN/OUT of each MapReduce job is through HDFS.
+
+In this example, we show how Hivemall deals with this problem. We use [KDD Cup 2012, Track 2 Task](https://github.com/myui/hivemall/wiki/KDDCup-2012-track-2-CTR-prediction-dataset) as an example.
+
+**WARNING**: rand_amplify() is supported in v0.2-beta1 and later.
+
+---
+# Amplify training examples in Map phase and shuffle them in Reduce phase
+Hivemall provides the **amplify** UDTF to enumerate iteration effects in machine learning without several MapReduce steps. 
+
+The amplify function returns multiple rows for each row.
+The first argument ${xtimes} is the multiplication factor.  
+In the following examples, the multiplication factor is set to 3.
+
+```sql
+set hivevar:xtimes=3;
+
+create or replace view training_x3
+as
+select 
+  * 
+from (
+select
+   amplify(${xtimes}, *) as (rowid, label, features)
+from  
+   training_orcfile
+) t
+CLUSTER BY rand();
+```
+
+In the above example, the  [CLUSTER BY](https://cwiki.apache.org/confluence/display/Hive/LanguageManual+SortBy#LanguageManualSortBy-SyntaxofClusterByandDistributeBy) clause distributes Map outputs to reducers using a random key for the distribution key. And then, the input records of the reducer is randomly shuffled.
+
+The multiplication of records and  the random shuffling has a similar effect to iterations.
+So, we recommend users to use an amplified view for training as follows:
+
+```sql
+create table lr_model_x3 
+as
+select 
+ feature,
+ cast(avg(weight) as float) as weight
+from 
+ (select 
+     logress(features,label) as (feature,weight)
+  from 
+     training_x3
+ ) t 
+group by feature;
+```
+
+The above query is executed by 2 MapReduce jobs as shown below:
+![amplifier](https://dl.dropboxusercontent.com/u/13123103/hivemall/amplify.png)
+[Here](https://dl.dropboxusercontent.com/u/13123103/hivemall/amplify_plan.txt) is the actual plan generated by the Hive.
+
+Using *trainning_x3*  instead of the plain training table results in higher and better AUC (0.746214) in [this](https://github.com/myui/hivemall/wiki/KDDCup-2012-track-2-CTR-prediction-(regression\)) example.
+
+A problem in amplify() is that the shuffle (copy) and merge phase of the stage 1 could become a bottleneck.
+When the training table is so large that involves 100 Map tasks, the merge operator needs to merge at least 100 files by (external) merge sort! 
+
+Note that the actual bottleneck is not M/R iterations but shuffling training instance. Iteration without shuffling (as in [the Spark example](http://spark.incubator.apache.org/examples.html)) causes very slow convergence and results in requiring more iterations. Shuffling cannot be avoided even in iterative MapReduce variants.
+
+![amplify elapsed](https://dl.dropboxusercontent.com/u/13123103/hivemall/amplify_elapsed.png)
+
+---
+# Amplify and shuffle training examples in each Map task
+
+To deal with large training data, Hivemall provides **rand_amplify** UDTF that randomly shuffles input rows in a Map task.
+The rand_amplify UDTF outputs rows in a random order when the local buffer specified by ${shufflebuffersize} is filled.
+
+With rand_amplify(), the view definition of training_x3 becomes as follows:
+```sql
+set hivevar:shufflebuffersize=1000;
+
+create or replace view training_x3
+as
+select
+   rand_amplify(${xtimes}, ${shufflebuffersize}, *) as (rowid, label, features)
+from  
+   training_orcfile;
+```
+
+The training query is executed as follows:
+![Random amplify](https://dl.dropboxusercontent.com/u/13123103/hivemall/randamplify.png)  
+[Here](https://dl.dropboxusercontent.com/u/13123103/hivemall/randamplify_plan.txt) is the actual query plan.
+
+The map-local multiplication and shuffling has no bottleneck in the merge phase and the query is efficiently executed within a single MapReduce job.
+
+![rand_amplify elapsed ](https://dl.dropboxusercontent.com/u/13123103/hivemall/randamplify_elapsed.png)
+
+Using *rand_amplify* results in a better AUC (0.743392) in [this](https://github.com/myui/hivemall/wiki/KDDCup-2012-track-2-CTR-prediction-(regression\)) example.
+
+---
+# Conclusion
+
+We recommend users to use *amplify()* for small training inputs and to use *rand_amplify()* for large training inputs to get a better accuracy in a reasonable training time.
+
+| Method     | ELAPSED TIME (sec) | AUC |
+|:-----------|--------------------|----:|
+| Plain | 89.718 | 0.734805 |
+| amplifier+clustered by | 479.855  | 0.746214 |
+| rand_amplifier | 116.424 | 0.743392 |
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/370e2aa3/docs/gitbook/resources/images/hivemall-logo-color-small.png
----------------------------------------------------------------------
diff --git a/docs/gitbook/resources/images/hivemall-logo-color-small.png b/docs/gitbook/resources/images/hivemall-logo-color-small.png
new file mode 100644
index 0000000..1194236
Binary files /dev/null and b/docs/gitbook/resources/images/hivemall-logo-color-small.png differ

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/370e2aa3/docs/gitbook/resources/images/techstack.png
----------------------------------------------------------------------
diff --git a/docs/gitbook/resources/images/techstack.png b/docs/gitbook/resources/images/techstack.png
new file mode 100644
index 0000000..fb29a3d
Binary files /dev/null and b/docs/gitbook/resources/images/techstack.png differ

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/370e2aa3/docs/gitbook/spark/.gitkeep
----------------------------------------------------------------------
diff --git a/docs/gitbook/spark/.gitkeep b/docs/gitbook/spark/.gitkeep
new file mode 100644
index 0000000..e69de29

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/370e2aa3/docs/gitbook/tips/README.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/tips/README.md b/docs/gitbook/tips/README.md
new file mode 100644
index 0000000..e69de29

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/370e2aa3/docs/gitbook/tips/addbias.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/tips/addbias.md b/docs/gitbook/tips/addbias.md
new file mode 100644
index 0000000..2b11d51
--- /dev/null
+++ b/docs/gitbook/tips/addbias.md
@@ -0,0 +1,43 @@
+A trainer learns the function f(x)=y, or weights _W_, of the following form to predict a label y where x is a feature vector.
+_y=f(x)=Wx_
+
+Without a bias clause (or regularization), f(x) cannot make a hyperplane that divides (1,1) and (2,2) becuase f(x) crosses the origin point (0,0).
+
+With bias clause b, a trainer learns the following f(x).
+_f(x)=Wx+b_ 
+Then, the predicted model considers bias existing in the dataset and the predicted hyperplane does not always cross the origin.
+
+**addBias()** of Hivemall, adds a bias to a feature vector. 
+To enable a bias clause, use addBias() for **both**_(important!)_ training and test data as follows.
+The bias _b_ is a feature of "0" ("-1" in before v0.3) by the default. See [AddBiasUDF](https://github.com/myui/hivemall/blob/master/src/main/hivemall/ftvec/AddBiasUDF.java) for the detail.
+
+Note that Bias is expressed as a feature that found in all training/testing examples.
+
+# Adding a bias clause to test data
+```sql
+create table e2006tfidf_test_exploded as
+select 
+  rowid,
+  target,
+  split(feature,":")[0] as feature,
+  cast(split(feature,":")[1] as float) as value
+  -- extract_feature(feature) as feature, -- hivemall v0.3.1 or later
+  -- extract_weight(feature) as value     -- hivemall v0.3.1 or later
+from 
+  e2006tfidf_test LATERAL VIEW explode(addBias(features)) t AS feature;
+```
+
+# Adding a bias clause to training data
+```
+create table e2006tfidf_pa1a_model as
+select 
+ feature,
+ avg(weight) as weight
+from 
+ (select 
+     pa1a_regress(addBias(features),target) as (feature,weight)
+  from 
+     e2006tfidf_train_x3
+ ) t 
+group by feature;
+```
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/370e2aa3/docs/gitbook/tips/emr.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/tips/emr.md b/docs/gitbook/tips/emr.md
new file mode 100644
index 0000000..4521635
--- /dev/null
+++ b/docs/gitbook/tips/emr.md
@@ -0,0 +1,182 @@
+## Prerequisite
+Learn how to use Hive with Elastic MapReduce (EMR).  
+http://docs.aws.amazon.com/ElasticMapReduce/latest/DeveloperGuide/emr-hive.html
+
+Before launching an EMR job, 
+* create ${s3bucket}/emr/outputs for outputs
+* optionally, create ${s3bucket}/emr/logs for logging
+* put [emr_hivemall_bootstrap.sh](https://raw.github.com/myui/hivemall/master/scripts/misc/emr_hivemall_bootstrap.sh) on ${s3bucket}/emr/conf
+
+Then, lunch an EMR job with hive in an interactive mode.
+I'm usually lunching EMR instances with cheap Spot instances through [CLI client](http://aws.amazon.com/developertools/2264) as follows:
+```
+./elastic-mapreduce --create --alive \
+ --name "Hive cluster" \
+ --hive-interactive --hive-versions latest \
+ --hive-site=s3://${s3bucket}/emr/conf/hive-site.xml \
+ --ami-version latest \
+ --instance-group master --instance-type m1.medium --instance-count 1 --bid-price 0.175 \
+ --instance-group core --instance-type m1.large --instance-count 3 --bid-price 0.35 \
+ --enable-debugging --log-uri s3n://${s3bucket}/emr/logs \
+ --bootstrap-action s3://elasticmapreduce/bootstrap-actions/run-if \
+   --args "instance.isMaster=true,s3://${s3bucket}/emr/conf/emr_hivemall_bootstrap.sh" --bootstrap-name "hivemall setup"
+ --bootstrap-action s3://elasticmapreduce/bootstrap-actions/install-ganglia --bootstrap-name "install ganglia"
+```
+_To use YARN instead of old Hadoop, specify "[--ami-version 3.0.0](http://docs.aws.amazon.com/ElasticMapReduce/latest/DeveloperGuide/emr-plan-ami.html#ami-versions-supported)". Hivemall works on both old Hadoop and YARN._
+
+Or, lunch an interactive EMR job using the EMR GUI wizard.
+![emr-wizard](https://dl.dropboxusercontent.com/u/13123103/emr-wizard.png)
+![emr-bootstrap](https://dl.dropboxusercontent.com/u/13123103/emr-bootstrap.png)
+
+## Data preparation
+
+Put training and test data in a TSV format on Amazon S3, e.g., on ${s3bucket}/datasets/news20b/[train|test].
+
+```sql
+create database news20;
+use news20;
+
+add jar ./tmp/hivemall.jar;
+source ./tmp/define-all.hive;
+
+set hivevar:s3bucket=YOUR_BUCKET_NAME;
+
+-- The default input split size is often too large for Hivemall
+set mapred.max.split.size=67108864;
+
+Create external table news20b_train (
+  rowid int,
+  label int,
+  features ARRAY<STRING>
+) ROW FORMAT DELIMITED FIELDS TERMINATED BY '\t' COLLECTION ITEMS TERMINATED BY "," 
+STORED AS TEXTFILE LOCATION 's3n://${s3bucket}/datasets/news20b/train';
+
+Create external table news20b_test (
+  rowid int, 
+  label int,
+  features ARRAY<STRING>
+) ROW FORMAT DELIMITED FIELDS TERMINATED BY '\t' COLLECTION ITEMS TERMINATED BY ","
+STORED AS TEXTFILE LOCATION 's3n://${s3bucket}/datasets/news20b/test';
+
+-- create or replace view news20b_train_x3
+-- as
+-- select 
+--  * 
+-- from (
+-- select
+--   amplify(3, *) as (rowid, label, features)
+-- from  
+--   news20b_train 
+-- ) t
+-- CLUSTER BY CAST(rand(47) * 100 as INT), CAST(rand(49) * 100 as INT), CAST(rand(50) * 100 as INT);
+
+create or replace view news20b_train_x3
+as
+select
+   rand_amplify(3, 1000, *) as (rowid, label, features)
+from  
+   news20b_train;
+
+create table news20b_test_exploded as
+select 
+  rowid,
+  label,
+  cast(split(feature,":")[0] as int) as feature,
+  cast(split(feature,":")[1] as float) as value
+from 
+  news20b_test LATERAL VIEW explode(addBias(features)) t AS feature;
+```
+
+---
+# Adaptive Regularization of Weight Vectors (AROW)
+
+## training
+```sql
+DROP TABLE news20b_arow_model1;
+CREATE EXTERNAL TABLE IF NOT EXISTS news20b_arow_model1 (
+  feature string,
+  weight float
+)
+ROW FORMAT DELIMITED 
+  FIELDS TERMINATED BY '\t'
+  LINES TERMINATED BY '\n'
+STORED AS TEXTFILE
+LOCATION 's3://${s3bucket}/emr/outputs/news20b_arow_model1';
+
+insert overwrite table news20b_arow_model1
+select 
+ feature,
+ cast(voted_avg(weight) as float) as weight
+from 
+ (select 
+     train_arow(addBias(features),label) as (feature,weight)
+  from 
+     news20b_train_x3
+ ) t 
+group by feature;
+```
+
+## prediction
+```sql
+create or replace view news20b_arow_predict1 
+as
+select
+  t.rowid, 
+  sum(m.weight * t.value) as total_weight,
+  case when sum(m.weight * t.value) > 0.0 then 1 else -1 end as label
+from 
+  news20b_test_exploded t LEFT OUTER JOIN
+  news20b_arow_model1 m ON (t.feature = m.feature)
+group by
+  t.rowid;
+```
+
+## evaluation
+```sql
+create or replace view news20b_arow_submit1 as
+select 
+  t.rowid, 
+  t.label as actual, 
+  pd.label as predicted
+from 
+  news20b_test t JOIN news20b_arow_predict1 pd 
+    on (t.rowid = pd.rowid);
+```
+
+```sql
+select count(1)/4996 from news20b_arow_submit1 
+where actual == predicted;
+```
+> 0.9659727782225781
+
+## Cleaning
+
+```sql
+drop table news20b_arow_model1;
+drop view news20b_arow_predict1;
+drop view news20b_arow_submit1;
+```
+
+---
+## Tips
+
+We recommended users to use m1.xlarge running Hivemall on EMR as follows.
+```
+./elastic-mapreduce --create --alive \
+ --name "Hive cluster" \
+ --hive-interactive --hive-versions latest \
+ --ami-version latest \
+ --instance-group master --instance-type m1.xlarge --instance-count 1 \
+ --instance-group core --instance-type m1.xlarge --instance-count 8 --bid-price 0.7 \
+ --instance-group task --instance-type m1.xlarge --instance-count 2 --bid-price 0.7 \
+ --enable-debugging --log-uri s3://mybucket/emr/logs \
+ --bootstrap-action s3://elasticmapreduce/bootstrap-actions/configure-hadoop \
+   --args "-m,mapred.child.java.opts=-Xmx1536m,-m,mapred.tasktracker.map.tasks.maximum=7,-m,mapred.tasktracker.reduce.tasks.maximum=2,-c,fs.s3n.multipart.uploads.enable=true,-c,fs.s3n.multipart.uploads.split.size=67108864" \
+ --bootstrap-action s3://elasticmapreduce/bootstrap-actions/run-if \
+   --args "instance.isMaster=true,s3://mybucket/emr/conf/emr_hivemall_bootstrap.sh" \
+   --bootstrap-name "hivemall setup" \
+ --bootstrap-action s3://elasticmapreduce/bootstrap-actions/install-ganglia \
+   --bootstrap-name "install ganglia" \
+ --availability-zone ap-northeast-1a
+```
+Using spot instance for core/task instance groups is the best way to save your money.
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/370e2aa3/docs/gitbook/tips/ensemble_learning.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/tips/ensemble_learning.md b/docs/gitbook/tips/ensemble_learning.md
new file mode 100644
index 0000000..6daaf1a
--- /dev/null
+++ b/docs/gitbook/tips/ensemble_learning.md
@@ -0,0 +1,180 @@
+This example explains how to run ensemble learning in Hivemall.   
+Two heads are better than one? Let's verify it by ensemble learning.
+
+<!-- toc -->
+
+---
+
+## UDF preparation
+```sql
+delete jar /home/myui/tmp/hivemall.jar;
+add jar /home/myui/tmp/hivemall.jar;
+
+source /home/myui/tmp/define-all.hive;
+```
+
+# [Case1] Model ensemble/mixing
+
+## training
+```sql
+SET hive.exec.parallel=true;
+SET hive.exec.parallel.thread.number=8;
+SET mapred.reduce.tasks=4;
+
+drop table news20mc_ensemble_model1;
+create table news20mc_ensemble_model1 as
+select 
+ label, 
+ -- cast(feature as int) as feature, -- hivemall v0.1
+ argmin_kld(feature, covar) as feature, -- hivemall v0.2 or later
+ voted_avg(weight) as weight
+from 
+ (select 
+     -- train_multiclass_cw(addBias(features),label) as (label,feature,weight)      -- hivemall v0.1
+     train_multiclass_cw(addBias(features),label) as (label,feature,weight,covar)   -- hivemall v0.2 or later
+  from 
+     news20mc_train_x3
+  union all
+  select 
+     -- train_multiclass_arow(addBias(features),label) as (label,feature,weight)    -- hivemall v0.1
+     train_multiclass_arow(addBias(features),label) as (label,feature,weight,covar) -- hivemall v0.2 or later
+  from 
+     news20mc_train_x3
+  union all
+  select 
+     -- train_multiclass_scw(addBias(features),label) as (label,feature,weight)     -- hivemall v0.1
+     train_multiclass_scw(addBias(features),label) as (label,feature,weight,covar)  -- hivemall v0.2 or later
+  from 
+     news20mc_train_x3
+ ) t 
+group by label, feature;
+
+-- reset to the default
+SET hive.exec.parallel=false;
+SET mapred.reduce.tasks=-1;
+```
+
+## prediction
+```sql
+create or replace view news20mc_ensemble_predict1 
+as
+select 
+  rowid, 
+  m.col0 as score, 
+  m.col1 as label
+from (
+select
+   rowid, 
+   maxrow(score, label) as m
+from (
+  select
+    t.rowid,
+    m.label,
+    sum(m.weight * t.value) as score
+  from 
+    news20mc_test_exploded t LEFT OUTER JOIN
+    news20mc_ensemble_model1 m ON (t.feature = m.feature)
+  group by
+    t.rowid, m.label
+) t1
+group by rowid
+) t2;
+```
+
+## evaluation
+```sql
+create or replace view news20mc_ensemble_submit1 as
+select 
+  t.label as actual, 
+  pd.label as predicted
+from 
+  news20mc_test t JOIN news20mc_ensemble_predict1 pd 
+    on (t.rowid = pd.rowid);
+```
+
+```
+select count(1)/3993 from news20mc_ensemble_submit1 
+where actual == predicted;
+```
+
+> 0.8494866015527173
+
+## Cleaning
+
+```sql
+drop table news20mc_ensemble_model1;
+drop view news20mc_ensemble_predict1;
+drop view news20mc_ensemble_submit1;
+```
+---
+
+Unfortunately, too many cooks spoil the broth in this case :-(
+
+| Algorithm | Accuracy |
+|:-----------|------------:|
+| AROW | 0.8474830954169797 |
+| SCW2 |  0.8482344102178813 |
+| Ensemble(model) | 0.8494866015527173 |
+| CW |  0.850488354620586 |
+
+
+---
+
+# [Case2] Prediction ensemble
+
+## prediction
+```sql
+create or replace view news20mc_pred_ensemble_predict1 
+as
+select 
+  rowid, 
+  m.col1 as label
+from (
+  select
+    rowid, 
+    maxrow(cnt, label) as m
+  from (
+    select
+      rowid,
+      label,
+      count(1) as cnt
+    from (
+      select * from news20mc_arow_predict1
+      union all
+      select * from news20mc_scw2_predict1
+      union all
+      select * from news20mc_cw_predict1
+    ) t1
+    group by rowid, label
+  ) t2
+  group by rowid
+) t3;
+```
+
+## evaluation
+```sql
+create or replace view news20mc_pred_ensemble_submit1 as
+select 
+  t.label as actual, 
+  pd.label as predicted
+from 
+  news20mc_test t JOIN news20mc_pred_ensemble_predict1 pd 
+    on (t.rowid = pd.rowid);
+```
+
+```
+select count(1)/3993 from news20mc_pred_ensemble_submit1 
+where actual == predicted;
+```
+
+> 0.8499874780866516
+
+Unfortunately, too many cooks spoil the broth in this case too :-(
+
+| Algorithm | Accuracy |
+|:-----------|------------:|
+| AROW | 0.8474830954169797 |
+| SCW2 |  0.8482344102178813 |
+| Ensemble(model) | 0.8494866015527173 |
+| Ensemble(prediction) | 0.8499874780866516 |
+| CW |  0.850488354620586 |
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/370e2aa3/docs/gitbook/tips/general_tips.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/tips/general_tips.md b/docs/gitbook/tips/general_tips.md
new file mode 100644
index 0000000..e69de29



[35/50] [abbrv] incubator-hivemall git commit: Merge branch 'master' into jira/HIVEMALL-12

Posted by my...@apache.org.
Merge branch 'master' into jira/HIVEMALL-12

# Conflicts:
#	src/site/markdown/contributing.md


Project: http://git-wip-us.apache.org/repos/asf/incubator-hivemall/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-hivemall/commit/02686099
Tree: http://git-wip-us.apache.org/repos/asf/incubator-hivemall/tree/02686099
Diff: http://git-wip-us.apache.org/repos/asf/incubator-hivemall/diff/02686099

Branch: refs/heads/master
Commit: 02686099341742210a01591743c916fc7bfc64c3
Parents: d433a0a 9a6d1c0
Author: myui <yu...@gmail.com>
Authored: Sat Nov 12 15:03:02 2016 +0900
Committer: myui <yu...@gmail.com>
Committed: Sat Nov 12 15:03:02 2016 +0900

----------------------------------------------------------------------
 bin/build_xgboost.sh              |  2 +-
 src/site/markdown/contributing.md | 18 ++++++++++++++++++
 2 files changed, 19 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/02686099/bin/build_xgboost.sh
----------------------------------------------------------------------
diff --cc bin/build_xgboost.sh
index 7dcd4f7,43ea515..9ba201f
--- a/bin/build_xgboost.sh
+++ b/bin/build_xgboost.sh
@@@ -1,24 -1,23 +1,24 @@@
  #!/bin/bash
 -
 -# Hivemall: Hive scalable Machine Learning Library
  #
 -# Copyright (C) 2015 Makoto YUI
 -# Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
 +# 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
  #
 -# Licensed 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
  #
 -#         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.
  #
 -# 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.
  
- set -eu
+ set -e
  set -o pipefail
  
  # Target commit hash value

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/02686099/src/site/markdown/contributing.md
----------------------------------------------------------------------
diff --cc src/site/markdown/contributing.md
index 999380a,17f7020..803d5f5
--- a/src/site/markdown/contributing.md
+++ b/src/site/markdown/contributing.md
@@@ -1,22 -1,21 +1,40 @@@
 +<!--
 +  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.
 +-->
 +
+ # How to build
+ 
+ #### Prerequisites
+ 
+ * Maven 3.x
+ * JDK 1.7 or later
+ 
+ ```
+ $ cd hivemall
+ 
+ # This is a workaround for resolving xgboost dependencies.
+ $ mvn validate -Pxgboost
+  
+ $ mvn clean package
+ ```
+ 
+ # Contribution guideline
+ 
  to appear ...
  
  [Here](http://www.apache.org/foundation/getinvolved.html) is a general guide for contributing to Apache Project.


[39/50] [abbrv] incubator-hivemall git commit: Updated README and removed obsolute files

Posted by my...@apache.org.
Updated README and removed obsolute files

Project: http://git-wip-us.apache.org/repos/asf/incubator-hivemall/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-hivemall/commit/32a657d2
Tree: http://git-wip-us.apache.org/repos/asf/incubator-hivemall/tree/32a657d2
Diff: http://git-wip-us.apache.org/repos/asf/incubator-hivemall/diff/32a657d2

Branch: refs/heads/master
Commit: 32a657d242cedc240132c4e4cb5b5b8f7926cd6e
Parents: fd872c2
Author: myui <yu...@gmail.com>
Authored: Sat Nov 12 16:21:11 2016 +0900
Committer: myui <yu...@gmail.com>
Committed: Sat Nov 12 16:21:11 2016 +0900

----------------------------------------------------------------------
 CONTRIBUTORS  | 55 ------------------------------------------
 ChangeLog.rst | 71 ------------------------------------------------------
 README.md     |  8 +++---
 3 files changed, 3 insertions(+), 131 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/32a657d2/CONTRIBUTORS
----------------------------------------------------------------------
diff --git a/CONTRIBUTORS b/CONTRIBUTORS
deleted file mode 100644
index a1d4648..0000000
--- a/CONTRIBUTORS
+++ /dev/null
@@ -1,55 +0,0 @@
- Project Lead
-==============
-
-* Makoto Yui
-
- Contributors
-==============
-
-* Kohei Ozaki
-	- Mavem pom
-	- Unit tests
-
-* Kohichi Miyano
-	- Hivemall's cute logo
-
-* Yukihiro Tagami
-	- Improvements on pom.xml and build.xml
-	- Fixed a bug/regression in Sha1UDFTest
-	- Fixed a bug in OnlineVariance and add OnlineVarianceTest
-
-* Ryu Kobayashi
-	- Fixed a bug in pom.xml
-
-* Jiro Nishiguchi
-	- Fixed a bug in pom.xml
-
-* Tsuyoshi Ozawa
-	- TravisCI integration
-	- Fixes on MixServerTest
-
-* Takahiko Ito
-        - Improvements on build.xml
-
-* Daniel Dai
-        - Pig support
-        
-* Kai Sasaki
-        - Minor fixes about travisCI and unit tests.
-
-* Ritta Narita
-        - Stack Machine implementation for RandomForest
-
-* Naoki Ishikawa
-	- Initial implementation of Factorization Machine
-
-* Takeshi Yamamuro
-	- Various enhancement to test cases
-	- Hivemall on Spark
-	- Xgboost integration
-
-* Arnab Rakshit
-	- R2 UDAF
-	
-* Sotaro Sugimoto
-        - Implementation of Field-aware Factorization Machines

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/32a657d2/ChangeLog.rst
----------------------------------------------------------------------
diff --git a/ChangeLog.rst b/ChangeLog.rst
deleted file mode 100644
index b0f3ddf..0000000
--- a/ChangeLog.rst
+++ /dev/null
@@ -1,71 +0,0 @@
-Version 0.2-alpha2 - to appear
--------------------------------
-
-* Enhancement
-    *
-* Bugfix
-    *
-
-Version 0.2-alpha1 - 2014/05/07
--------------------------------
-
-* Enhancement
-    * kNN search using Minhash
-    * kNN search using b-Bit Minhash
-    * kNN search using cosine similarity
-    * Introduced a recommendation scheme using Minhash
-    * Modified to use OpenHashTable instead of HashMap for reducing memory consumption [9698972]
-* Bugfix
-    * Fixed a bug in rand_amplify() [0962c07]
-
-Version 0.2-beta1 - 2013/11/22
------------------------------
-* Enhancement
-  * Added rand_amplify() [538d866, 23b4f3b]
-  * Added EMR bootstrap for hivemall [2a834e8]
-  * Added AROW with a hinge loss [ae5c100]
-* Bugfix
-  * Fixed a bug in maven pom.xml [0d33ed7]
-
-Version 0.1     - 2013/10/25
-----------------------------
-
-* Enhancement
-    * Added AROW regression [f2f00a2]
-    * Added AROW with a hinge loss (arowh_regress()) [239e90d]
-
-* Bugfix
-    * Fixed a bug of null feature handling in classification/regression [1f392a3]
-
-Version 0.1-rc4 - 2013/10/18
-----------------------------
-
-* Enhancement
-    * Added a function prefixed_hash_values() [0a6ffb3]
-
-* Bugfix
-    * Fixed recursion in OnlineVariance#mean() [c5b8c5b]
-    * Fixed score calculation w.r.t bias values in predict() [f895269, 0130cfe]
-
-Version 0.1-rc3 - 2013/10/08
-----------------------------
-
-* Enhancement
-    * Add new classifiers (Confidence Weighted, AROW, Soft Confidence Weighted)
-
-* Bugfix
-    * fixed a bug in PA1a and PA2a that stddev was not calculated correctly [75ccdd336c]
-    * fixed option handle for aggressive parameter C [095d9395f0]
-    * changed the default power_t from 0.25 to 0.1 [7081291]
-
-Version 0.1-rc2 - 2013/10/04 
-----------------------------
-
-* Bugfix
-    * removed a dependency to serde.Constants for Hive 0.10 and later [054b9b8022]
-    * fixed a serious bug in logress() [fb50235268]
-
-Version 0.1-rc1 - 2013/10/02
-----------------------------
-
-This is the first release. Hello Hivemall!

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/32a657d2/README.md
----------------------------------------------------------------------
diff --git a/README.md b/README.md
index 1aa9fea..2841ef7 100644
--- a/README.md
+++ b/README.md
@@ -31,10 +31,8 @@ Apache Hivemall: Hive scalable machine learning library
 
 Hivemall is a scalable machine learning library that runs on Apache Hive, Apache Spark, and Apache Pig. Hivemall is designed to be scalable to the number of training instances as well as the number of training features.
 
-![logo](https://raw.github.com/myui/hivemall/master/resources/hivemall-logo-color-small.png "Hivemall's cute(!?) logo")
-
-Basic Usage
-------------
+Usage
+-----
 
 [![Hivemall](https://gist.githubusercontent.com/myui/d29241262f9313dec706/raw/caead313efd829b42a4a4183285e8b53cf26ab62/hadoopsummit14_slideshare.png)](http://www.slideshare.net/myui/hadoopsummit16-myui)
 
@@ -43,4 +41,4 @@ Find more examples on [our user guide](http://hivemall.incubator.apache.org/user
 Support
 -------
 
-Support is through the [user@hivemall.incubator.apache.org](http://hivemall.incubator.apache.org/mail-lists.html), not by a direct e-mail. 
+Support is through [user@hivemall.incubator.apache.org](http://hivemall.incubator.apache.org/mail-lists.html), not by a direct e-mail. 


[42/50] [abbrv] incubator-hivemall git commit: Added missing header

Posted by my...@apache.org.
Added missing header

Project: http://git-wip-us.apache.org/repos/asf/incubator-hivemall/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-hivemall/commit/e987711c
Tree: http://git-wip-us.apache.org/repos/asf/incubator-hivemall/tree/e987711c
Diff: http://git-wip-us.apache.org/repos/asf/incubator-hivemall/diff/e987711c

Branch: refs/heads/master
Commit: e987711cd09df8bb95157834b713469089e6d689
Parents: caeafae
Author: myui <yu...@gmail.com>
Authored: Mon Nov 14 20:15:41 2016 +0900
Committer: myui <yu...@gmail.com>
Committed: Mon Nov 14 20:15:41 2016 +0900

----------------------------------------------------------------------
 docs/gitbook/FOOTER.md | 19 +++++++++++++++++++
 1 file changed, 19 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/e987711c/docs/gitbook/FOOTER.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/FOOTER.md b/docs/gitbook/FOOTER.md
index 588afbb..b6f2c55 100644
--- a/docs/gitbook/FOOTER.md
+++ b/docs/gitbook/FOOTER.md
@@ -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.
+-->
+
 <sub><font color="gray">
 Apache Hivemall is an effort undergoing incubation at The Apache Software Foundation (ASF), sponsored by the Apache Incubator.
 </font></sub>
\ No newline at end of file


[49/50] [abbrv] incubator-hivemall git commit: Update README.md

Posted by my...@apache.org.
Update README.md

Updated links

Project: http://git-wip-us.apache.org/repos/asf/incubator-hivemall/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-hivemall/commit/69496fa1
Tree: http://git-wip-us.apache.org/repos/asf/incubator-hivemall/tree/69496fa1
Diff: http://git-wip-us.apache.org/repos/asf/incubator-hivemall/diff/69496fa1

Branch: refs/heads/master
Commit: 69496fa1a099737fa4aa4044d97870d4a7513361
Parents: 3e96470
Author: Makoto YUI <yu...@gmail.com>
Authored: Mon Nov 21 17:24:24 2016 +0900
Committer: GitHub <no...@github.com>
Committed: Mon Nov 21 17:24:24 2016 +0900

----------------------------------------------------------------------
 README.md | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/69496fa1/README.md
----------------------------------------------------------------------
diff --git a/README.md b/README.md
index 7a0978b..fb61abc 100644
--- a/README.md
+++ b/README.md
@@ -21,7 +21,7 @@ Apache Hivemall: Hive scalable machine learning library
 =======================================================
 [![Build Status](https://travis-ci.org/myui/hivemall.svg?branch=master)](https://travis-ci.org/myui/hivemall)
 [![Coversity](https://scan.coverity.com/projects/4549/badge.svg)](https://scan.coverity.com/projects/4549)
-[![Documentation Status](https://readthedocs.org/projects/hivemall-docs/badge/?version=latest)](https://readthedocs.org/projects/hivemall-docs/?badge=latest)
+[![Documentation Status](https://img.shields.io/:docs-latest-green.svg)](http://hivemall.incubator.apache.org/userguide/)
 [![License](http://img.shields.io/:license-Apache_v2-blue.svg)](https://github.com/myui/hivemall/blob/master/LICENSE)
 [![Coverage Status](https://coveralls.io/repos/github/myui/hivemall/badge.svg)](https://coveralls.io/github/myui/hivemall)
 [![Twitter Follow](https://img.shields.io/twitter/follow/ApacheHivemall.svg?style=social&label=Follow)](https://twitter.com/ApacheHivemall)
@@ -35,7 +35,7 @@ Hivemall is a scalable machine learning library that runs on Apache Hive, Apache
 Usage
 -----
 
-[![Hivemall](https://gist.githubusercontent.com/myui/d29241262f9313dec706/raw/caead313efd829b42a4a4183285e8b53cf26ab62/hadoopsummit14_slideshare.png)](http://www.slideshare.net/myui/hadoopsummit16-myui)
+[![Hivemall](https://gist.githubusercontent.com/myui/d29241262f9313dec706/raw/caead313efd829b42a4a4183285e8b53cf26ab62/hadoopsummit14_slideshare.png)](http://www.slideshare.net/myui/dots20161029-myui/7)
 
 Find more examples on [our user guide](http://hivemall.incubator.apache.org/userguide/index.html) and find a brief introduction to Hivemall in [this slide](http://www.slideshare.net/myui/hadoopsummit16-myui).
 


[16/50] [abbrv] incubator-hivemall git commit: Updated license headers

Posted by my...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/utils/collections/LRUMap.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/utils/collections/LRUMap.java b/core/src/main/java/hivemall/utils/collections/LRUMap.java
index 9e9cdde..bfae4d7 100644
--- a/core/src/main/java/hivemall/utils/collections/LRUMap.java
+++ b/core/src/main/java/hivemall/utils/collections/LRUMap.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.utils.collections;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/utils/collections/OpenHashMap.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/utils/collections/OpenHashMap.java b/core/src/main/java/hivemall/utils/collections/OpenHashMap.java
index 1d9d608..b1f5765 100644
--- a/core/src/main/java/hivemall/utils/collections/OpenHashMap.java
+++ b/core/src/main/java/hivemall/utils/collections/OpenHashMap.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.
  */
 //
 //   Copyright (C) 2010 catchpole.net

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/utils/collections/OpenHashTable.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/utils/collections/OpenHashTable.java b/core/src/main/java/hivemall/utils/collections/OpenHashTable.java
index 4ea83c4..1a3dff7 100644
--- a/core/src/main/java/hivemall/utils/collections/OpenHashTable.java
+++ b/core/src/main/java/hivemall/utils/collections/OpenHashTable.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.utils.collections;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/utils/collections/RingBuffer.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/utils/collections/RingBuffer.java b/core/src/main/java/hivemall/utils/collections/RingBuffer.java
index cab50e4..ff77e71 100644
--- a/core/src/main/java/hivemall/utils/collections/RingBuffer.java
+++ b/core/src/main/java/hivemall/utils/collections/RingBuffer.java
@@ -1,19 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.utils.collections;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/utils/collections/SparseIntArray.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/utils/collections/SparseIntArray.java b/core/src/main/java/hivemall/utils/collections/SparseIntArray.java
index ef84451..2c60258 100644
--- a/core/src/main/java/hivemall/utils/collections/SparseIntArray.java
+++ b/core/src/main/java/hivemall/utils/collections/SparseIntArray.java
@@ -1,19 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.utils.collections;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/utils/concurrent/ExecutorFactory.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/utils/concurrent/ExecutorFactory.java b/core/src/main/java/hivemall/utils/concurrent/ExecutorFactory.java
index 2a4e142..6d6b33b 100644
--- a/core/src/main/java/hivemall/utils/concurrent/ExecutorFactory.java
+++ b/core/src/main/java/hivemall/utils/concurrent/ExecutorFactory.java
@@ -1,19 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.utils.concurrent;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/utils/concurrent/NamedThreadFactory.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/utils/concurrent/NamedThreadFactory.java b/core/src/main/java/hivemall/utils/concurrent/NamedThreadFactory.java
index 470d092..5676699 100644
--- a/core/src/main/java/hivemall/utils/concurrent/NamedThreadFactory.java
+++ b/core/src/main/java/hivemall/utils/concurrent/NamedThreadFactory.java
@@ -1,19 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.utils.concurrent;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/utils/datetime/DateTimeFormatter.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/utils/datetime/DateTimeFormatter.java b/core/src/main/java/hivemall/utils/datetime/DateTimeFormatter.java
index 0fdb1bf..cf05ce1 100644
--- a/core/src/main/java/hivemall/utils/datetime/DateTimeFormatter.java
+++ b/core/src/main/java/hivemall/utils/datetime/DateTimeFormatter.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.utils.datetime;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/utils/datetime/StopWatch.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/utils/datetime/StopWatch.java b/core/src/main/java/hivemall/utils/datetime/StopWatch.java
index 40fb9c9..ce0c011 100644
--- a/core/src/main/java/hivemall/utils/datetime/StopWatch.java
+++ b/core/src/main/java/hivemall/utils/datetime/StopWatch.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.utils.datetime;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/utils/hadoop/HadoopUtils.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/utils/hadoop/HadoopUtils.java b/core/src/main/java/hivemall/utils/hadoop/HadoopUtils.java
index 76f91fd..10a17dc 100644
--- a/core/src/main/java/hivemall/utils/hadoop/HadoopUtils.java
+++ b/core/src/main/java/hivemall/utils/hadoop/HadoopUtils.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.utils.hadoop;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/utils/hadoop/HiveUtils.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/utils/hadoop/HiveUtils.java b/core/src/main/java/hivemall/utils/hadoop/HiveUtils.java
index 63c3c8e..d8b1aef 100644
--- a/core/src/main/java/hivemall/utils/hadoop/HiveUtils.java
+++ b/core/src/main/java/hivemall/utils/hadoop/HiveUtils.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.utils.hadoop;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/utils/hadoop/Text2.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/utils/hadoop/Text2.java b/core/src/main/java/hivemall/utils/hadoop/Text2.java
index 1261c77..efff276 100644
--- a/core/src/main/java/hivemall/utils/hadoop/Text2.java
+++ b/core/src/main/java/hivemall/utils/hadoop/Text2.java
@@ -1,19 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.utils.hadoop;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/utils/hadoop/Text3.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/utils/hadoop/Text3.java b/core/src/main/java/hivemall/utils/hadoop/Text3.java
index d4f3702..bce583e 100644
--- a/core/src/main/java/hivemall/utils/hadoop/Text3.java
+++ b/core/src/main/java/hivemall/utils/hadoop/Text3.java
@@ -1,19 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.utils.hadoop;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/utils/hadoop/WritableUtils.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/utils/hadoop/WritableUtils.java b/core/src/main/java/hivemall/utils/hadoop/WritableUtils.java
index c92f6ce..a4f2691 100644
--- a/core/src/main/java/hivemall/utils/hadoop/WritableUtils.java
+++ b/core/src/main/java/hivemall/utils/hadoop/WritableUtils.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.utils.hadoop;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/utils/hashing/HashFunction.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/utils/hashing/HashFunction.java b/core/src/main/java/hivemall/utils/hashing/HashFunction.java
index 8250def..eb44915 100644
--- a/core/src/main/java/hivemall/utils/hashing/HashFunction.java
+++ b/core/src/main/java/hivemall/utils/hashing/HashFunction.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.utils.hashing;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/utils/hashing/HashFunctionFactory.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/utils/hashing/HashFunctionFactory.java b/core/src/main/java/hivemall/utils/hashing/HashFunctionFactory.java
index fe7c0d2..bc4339f 100644
--- a/core/src/main/java/hivemall/utils/hashing/HashFunctionFactory.java
+++ b/core/src/main/java/hivemall/utils/hashing/HashFunctionFactory.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.utils.hashing;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/utils/hashing/MurmurHash3.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/utils/hashing/MurmurHash3.java b/core/src/main/java/hivemall/utils/hashing/MurmurHash3.java
index 702c4fd..4819deb 100644
--- a/core/src/main/java/hivemall/utils/hashing/MurmurHash3.java
+++ b/core/src/main/java/hivemall/utils/hashing/MurmurHash3.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.utils.hashing;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/utils/hashing/MurmurHash3Function.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/utils/hashing/MurmurHash3Function.java b/core/src/main/java/hivemall/utils/hashing/MurmurHash3Function.java
index aa5c826..e0923ef 100644
--- a/core/src/main/java/hivemall/utils/hashing/MurmurHash3Function.java
+++ b/core/src/main/java/hivemall/utils/hashing/MurmurHash3Function.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.utils.hashing;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/utils/io/ASCII85InputStream.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/utils/io/ASCII85InputStream.java b/core/src/main/java/hivemall/utils/io/ASCII85InputStream.java
index 49a5f53..bc0ca16 100644
--- a/core/src/main/java/hivemall/utils/io/ASCII85InputStream.java
+++ b/core/src/main/java/hivemall/utils/io/ASCII85InputStream.java
@@ -1,19 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.utils.io;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/utils/io/ASCII85OutputStream.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/utils/io/ASCII85OutputStream.java b/core/src/main/java/hivemall/utils/io/ASCII85OutputStream.java
index bc2ac9c..fc7d440 100644
--- a/core/src/main/java/hivemall/utils/io/ASCII85OutputStream.java
+++ b/core/src/main/java/hivemall/utils/io/ASCII85OutputStream.java
@@ -1,19 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.utils.io;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/utils/io/Base91InputStream.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/utils/io/Base91InputStream.java b/core/src/main/java/hivemall/utils/io/Base91InputStream.java
index 1fcb3be..9ff78b3 100644
--- a/core/src/main/java/hivemall/utils/io/Base91InputStream.java
+++ b/core/src/main/java/hivemall/utils/io/Base91InputStream.java
@@ -1,19 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.utils.io;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/utils/io/Base91OutputStream.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/utils/io/Base91OutputStream.java b/core/src/main/java/hivemall/utils/io/Base91OutputStream.java
index 0f29b59..71f9459 100644
--- a/core/src/main/java/hivemall/utils/io/Base91OutputStream.java
+++ b/core/src/main/java/hivemall/utils/io/Base91OutputStream.java
@@ -1,19 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.utils.io;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/utils/io/CompressionStreamFactory.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/utils/io/CompressionStreamFactory.java b/core/src/main/java/hivemall/utils/io/CompressionStreamFactory.java
index 7d8f7af..2127e7f 100644
--- a/core/src/main/java/hivemall/utils/io/CompressionStreamFactory.java
+++ b/core/src/main/java/hivemall/utils/io/CompressionStreamFactory.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.utils.io;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/utils/io/DeflaterOutputStream.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/utils/io/DeflaterOutputStream.java b/core/src/main/java/hivemall/utils/io/DeflaterOutputStream.java
index b31d5eb..b4cc1e2 100644
--- a/core/src/main/java/hivemall/utils/io/DeflaterOutputStream.java
+++ b/core/src/main/java/hivemall/utils/io/DeflaterOutputStream.java
@@ -1,19 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.utils.io;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/utils/io/FastByteArrayInputStream.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/utils/io/FastByteArrayInputStream.java b/core/src/main/java/hivemall/utils/io/FastByteArrayInputStream.java
index 76dacab..d141abf 100644
--- a/core/src/main/java/hivemall/utils/io/FastByteArrayInputStream.java
+++ b/core/src/main/java/hivemall/utils/io/FastByteArrayInputStream.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.utils.io;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/utils/io/FastByteArrayOutputStream.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/utils/io/FastByteArrayOutputStream.java b/core/src/main/java/hivemall/utils/io/FastByteArrayOutputStream.java
index 591d83a..03cd9a5 100644
--- a/core/src/main/java/hivemall/utils/io/FastByteArrayOutputStream.java
+++ b/core/src/main/java/hivemall/utils/io/FastByteArrayOutputStream.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.utils.io;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/utils/io/FastMultiByteArrayOutputStream.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/utils/io/FastMultiByteArrayOutputStream.java b/core/src/main/java/hivemall/utils/io/FastMultiByteArrayOutputStream.java
index 9c7965f..6685774 100644
--- a/core/src/main/java/hivemall/utils/io/FastMultiByteArrayOutputStream.java
+++ b/core/src/main/java/hivemall/utils/io/FastMultiByteArrayOutputStream.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.utils.io;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/utils/io/FileUtils.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/utils/io/FileUtils.java b/core/src/main/java/hivemall/utils/io/FileUtils.java
index 4e5bb99..a6b2e35 100644
--- a/core/src/main/java/hivemall/utils/io/FileUtils.java
+++ b/core/src/main/java/hivemall/utils/io/FileUtils.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.utils.io;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/utils/io/FinishableOutputStream.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/utils/io/FinishableOutputStream.java b/core/src/main/java/hivemall/utils/io/FinishableOutputStream.java
index fef288e..e921167 100644
--- a/core/src/main/java/hivemall/utils/io/FinishableOutputStream.java
+++ b/core/src/main/java/hivemall/utils/io/FinishableOutputStream.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.utils.io;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/utils/io/FinishableOutputStreamAdapter.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/utils/io/FinishableOutputStreamAdapter.java b/core/src/main/java/hivemall/utils/io/FinishableOutputStreamAdapter.java
index 19b866e..400294d 100644
--- a/core/src/main/java/hivemall/utils/io/FinishableOutputStreamAdapter.java
+++ b/core/src/main/java/hivemall/utils/io/FinishableOutputStreamAdapter.java
@@ -1,19 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.utils.io;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/utils/io/IOUtils.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/utils/io/IOUtils.java b/core/src/main/java/hivemall/utils/io/IOUtils.java
index 1802dfc..919fe17 100644
--- a/core/src/main/java/hivemall/utils/io/IOUtils.java
+++ b/core/src/main/java/hivemall/utils/io/IOUtils.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.utils.io;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/utils/io/NIOUtils.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/utils/io/NIOUtils.java b/core/src/main/java/hivemall/utils/io/NIOUtils.java
index 71c3505..acd1783 100644
--- a/core/src/main/java/hivemall/utils/io/NIOUtils.java
+++ b/core/src/main/java/hivemall/utils/io/NIOUtils.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.utils.io;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/utils/io/NioFixedSegment.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/utils/io/NioFixedSegment.java b/core/src/main/java/hivemall/utils/io/NioFixedSegment.java
index 21eb1e1..34c16ae 100644
--- a/core/src/main/java/hivemall/utils/io/NioFixedSegment.java
+++ b/core/src/main/java/hivemall/utils/io/NioFixedSegment.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.utils.io;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/utils/io/NioSegment.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/utils/io/NioSegment.java b/core/src/main/java/hivemall/utils/io/NioSegment.java
index 018dfdf..59bd514 100644
--- a/core/src/main/java/hivemall/utils/io/NioSegment.java
+++ b/core/src/main/java/hivemall/utils/io/NioSegment.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.utils.io;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/utils/io/NioStatefullSegment.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/utils/io/NioStatefullSegment.java b/core/src/main/java/hivemall/utils/io/NioStatefullSegment.java
index 25bcf06..5074dca 100644
--- a/core/src/main/java/hivemall/utils/io/NioStatefullSegment.java
+++ b/core/src/main/java/hivemall/utils/io/NioStatefullSegment.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.utils.io;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/utils/io/Segments.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/utils/io/Segments.java b/core/src/main/java/hivemall/utils/io/Segments.java
index 62fb648..bee7b1c 100644
--- a/core/src/main/java/hivemall/utils/io/Segments.java
+++ b/core/src/main/java/hivemall/utils/io/Segments.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.utils.io;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/utils/lang/ArrayUtils.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/utils/lang/ArrayUtils.java b/core/src/main/java/hivemall/utils/lang/ArrayUtils.java
index 5de1155..521dcbd 100644
--- a/core/src/main/java/hivemall/utils/lang/ArrayUtils.java
+++ b/core/src/main/java/hivemall/utils/lang/ArrayUtils.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.utils.lang;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/utils/lang/BitUtils.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/utils/lang/BitUtils.java b/core/src/main/java/hivemall/utils/lang/BitUtils.java
index f6f2af8..5f4235e 100644
--- a/core/src/main/java/hivemall/utils/lang/BitUtils.java
+++ b/core/src/main/java/hivemall/utils/lang/BitUtils.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.utils.lang;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/utils/lang/CommandLineUtils.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/utils/lang/CommandLineUtils.java b/core/src/main/java/hivemall/utils/lang/CommandLineUtils.java
index c6a0772..61e844e 100644
--- a/core/src/main/java/hivemall/utils/lang/CommandLineUtils.java
+++ b/core/src/main/java/hivemall/utils/lang/CommandLineUtils.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.utils.lang;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/utils/lang/Copyable.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/utils/lang/Copyable.java b/core/src/main/java/hivemall/utils/lang/Copyable.java
index a5b044d..827a2ca 100644
--- a/core/src/main/java/hivemall/utils/lang/Copyable.java
+++ b/core/src/main/java/hivemall/utils/lang/Copyable.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.utils.lang;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/utils/lang/Counter.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/utils/lang/Counter.java b/core/src/main/java/hivemall/utils/lang/Counter.java
index b864ccd..76e8b9b 100644
--- a/core/src/main/java/hivemall/utils/lang/Counter.java
+++ b/core/src/main/java/hivemall/utils/lang/Counter.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.utils.lang;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/utils/lang/FloatAccumulator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/utils/lang/FloatAccumulator.java b/core/src/main/java/hivemall/utils/lang/FloatAccumulator.java
index 8e3d8a6..6a48c01 100644
--- a/core/src/main/java/hivemall/utils/lang/FloatAccumulator.java
+++ b/core/src/main/java/hivemall/utils/lang/FloatAccumulator.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.utils.lang;
 



[05/50] [abbrv] incubator-hivemall git commit: Merge pull request #376 from myui/gitbook

Posted by my...@apache.org.
Merge pull request #376 from myui/gitbook

Added a gitbook userguide

Project: http://git-wip-us.apache.org/repos/asf/incubator-hivemall/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-hivemall/commit/ef13a62a
Tree: http://git-wip-us.apache.org/repos/asf/incubator-hivemall/tree/ef13a62a
Diff: http://git-wip-us.apache.org/repos/asf/incubator-hivemall/diff/ef13a62a

Branch: refs/heads/master
Commit: ef13a62a2eefe8e4b6564eb132b2cdcdf2a8d5ac
Parents: d3d7b55 370e2aa
Author: Makoto YUI <yu...@gmail.com>
Authored: Sun Oct 23 04:30:33 2016 +0900
Committer: GitHub <no...@github.com>
Committed: Sun Oct 23 04:30:33 2016 +0900

----------------------------------------------------------------------
 docs/gitbook/.gitignore                         |   3 +
 docs/gitbook/README.md                          |  22 +
 docs/gitbook/SUMMARY.md                         | 129 +++++
 docs/gitbook/anomaly/lof.md                     | 185 ++++++
 docs/gitbook/binaryclass/a9a.md                 |   0
 docs/gitbook/binaryclass/a9a_dataset.md         |  43 ++
 docs/gitbook/binaryclass/a9a_lr.md              |  79 +++
 docs/gitbook/binaryclass/a9a_minibatch.md       |  37 ++
 docs/gitbook/binaryclass/kdd2010a.md            |   0
 docs/gitbook/binaryclass/kdd2010a_dataset.md    |  72 +++
 docs/gitbook/binaryclass/kdd2010a_scw.md        | 185 ++++++
 docs/gitbook/binaryclass/kdd2010b.md            |   0
 docs/gitbook/binaryclass/kdd2010b_arow.md       |  51 ++
 docs/gitbook/binaryclass/kdd2010b_dataset.md    |  59 ++
 docs/gitbook/binaryclass/news20.md              |   0
 docs/gitbook/binaryclass/news20_adagrad.md      | 170 ++++++
 docs/gitbook/binaryclass/news20_dataset.md      |  84 +++
 docs/gitbook/binaryclass/news20_pa.md           | 243 ++++++++
 docs/gitbook/binaryclass/news20_scw.md          | 272 +++++++++
 docs/gitbook/binaryclass/webspam.md             |   0
 docs/gitbook/binaryclass/webspam_dataset.md     |  76 +++
 docs/gitbook/binaryclass/webspam_scw.md         | 136 +++++
 docs/gitbook/book.json                          |  64 +++
 docs/gitbook/eval/datagen.md                    |   0
 docs/gitbook/eval/lr_datagen.md                 |  92 +++
 docs/gitbook/eval/stat_eval.md                  |  56 ++
 docs/gitbook/ft_engineering/ft_trans.md         |   0
 docs/gitbook/ft_engineering/hashing.md          | 103 ++++
 docs/gitbook/ft_engineering/quantify.md         | 164 ++++++
 docs/gitbook/ft_engineering/scaling.md          | 173 ++++++
 docs/gitbook/ft_engineering/tfidf.md            | 149 +++++
 docs/gitbook/ft_engineering/vectorizer.md       |  42 ++
 docs/gitbook/getting_started/README.md          |   1 +
 docs/gitbook/getting_started/input-format.md    | 214 +++++++
 docs/gitbook/getting_started/installation.md    |  25 +
 .../getting_started/permanent-functions.md      |  42 ++
 docs/gitbook/misc/generic_funcs.md              | 210 +++++++
 docs/gitbook/misc/tokenizer.md                  |  30 +
 docs/gitbook/misc/topk.md                       | 288 ++++++++++
 docs/gitbook/multiclass/iris.md                 |   0
 docs/gitbook/multiclass/iris_dataset.md         | 203 +++++++
 docs/gitbook/multiclass/iris_randomforest.md    | 307 ++++++++++
 docs/gitbook/multiclass/iris_scw.md             | 307 ++++++++++
 docs/gitbook/multiclass/news20.md               |   0
 docs/gitbook/multiclass/news20_dataset.md       |  77 +++
 docs/gitbook/multiclass/news20_ensemble.md      | 180 ++++++
 .../multiclass/news20_one-vs-the-rest.md        | 330 +++++++++++
 .../news20_one-vs-the-rest_dataset.md           |  52 ++
 docs/gitbook/multiclass/news20_pa.md            |  90 +++
 docs/gitbook/multiclass/news20_scw.md           | 319 +++++++++++
 docs/gitbook/pig/.gitkeep                       |   0
 docs/gitbook/recommend/cf.md                    |   0
 docs/gitbook/recommend/item_based_cf.md         | 565 +++++++++++++++++++
 docs/gitbook/recommend/movielens.md             |   0
 docs/gitbook/recommend/movielens_cv.md          |  63 +++
 docs/gitbook/recommend/movielens_dataset.md     | 160 ++++++
 docs/gitbook/recommend/movielens_fm.md          | 249 ++++++++
 docs/gitbook/recommend/movielens_mf.md          | 137 +++++
 docs/gitbook/recommend/news20.md                |   0
 docs/gitbook/recommend/news20_bbit_minhash.md   |  50 ++
 docs/gitbook/recommend/news20_jaccard.md        | 123 ++++
 docs/gitbook/recommend/news20_knn.md            | 103 ++++
 docs/gitbook/regression/e2006.md                |   0
 docs/gitbook/regression/e2006_arow.md           | 259 +++++++++
 docs/gitbook/regression/e2006_dataset.md        |  72 +++
 docs/gitbook/regression/kddcup12tr2.md          |   0
 docs/gitbook/regression/kddcup12tr2_adagrad.md  | 109 ++++
 docs/gitbook/regression/kddcup12tr2_dataset.md  | 227 ++++++++
 docs/gitbook/regression/kddcup12tr2_lr.md       | 141 +++++
 .../regression/kddcup12tr2_lr_amplify.md        | 103 ++++
 .../images/hivemall-logo-color-small.png        | Bin 0 -> 33630 bytes
 docs/gitbook/resources/images/techstack.png     | Bin 0 -> 198979 bytes
 docs/gitbook/spark/.gitkeep                     |   0
 docs/gitbook/tips/README.md                     |   0
 docs/gitbook/tips/addbias.md                    |  43 ++
 docs/gitbook/tips/emr.md                        | 182 ++++++
 docs/gitbook/tips/ensemble_learning.md          | 180 ++++++
 docs/gitbook/tips/general_tips.md               |   0
 docs/gitbook/tips/hadoop_tuning.md              |  79 +++
 docs/gitbook/tips/mixserver.md                  |  68 +++
 docs/gitbook/tips/rand_amplify.md               | 103 ++++
 docs/gitbook/tips/rowid.md                      |  31 +
 docs/gitbook/tips/rt_prediction.md              | 234 ++++++++
 docs/gitbook/troubleshooting/README.md          |   0
 docs/gitbook/troubleshooting/asterisk.md        |   3 +
 .../troubleshooting/mapjoin_classcastex.md      |   8 +
 .../troubleshooting/mapjoin_task_error.md       |   8 +
 docs/gitbook/troubleshooting/num_mappers.md     |  20 +
 docs/gitbook/troubleshooting/oom.md             |  20 +
 src/site/site.xml                               |   6 +-
 src/site/xdoc/index.xml.vm                      |   7 +-
 91 files changed, 8711 insertions(+), 6 deletions(-)
----------------------------------------------------------------------



[08/50] [abbrv] incubator-hivemall git commit: Updated links

Posted by my...@apache.org.
Updated links


Project: http://git-wip-us.apache.org/repos/asf/incubator-hivemall/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-hivemall/commit/77d783ad
Tree: http://git-wip-us.apache.org/repos/asf/incubator-hivemall/tree/77d783ad
Diff: http://git-wip-us.apache.org/repos/asf/incubator-hivemall/diff/77d783ad

Branch: refs/heads/master
Commit: 77d783adefe2400ffffed4841ef99b88b3b515d0
Parents: 34c2b0b
Author: myui <yu...@gmail.com>
Authored: Sun Oct 23 04:45:55 2016 +0900
Committer: myui <yu...@gmail.com>
Committed: Sun Oct 23 04:45:55 2016 +0900

----------------------------------------------------------------------
 README.md         | 4 ++--
 src/site/site.xml | 1 +
 2 files changed, 3 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/77d783ad/README.md
----------------------------------------------------------------------
diff --git a/README.md b/README.md
index 66e72d4..75ad6cd 100644
--- a/README.md
+++ b/README.md
@@ -6,8 +6,8 @@ Hivemall: Hive scalable machine learning library
 [![License](http://img.shields.io/:license-Apache_v2-blue.svg)](https://github.com/myui/hivemall/blob/master/LICENSE)
 [![Coverage Status](https://coveralls.io/repos/github/myui/hivemall/badge.svg)](https://coveralls.io/github/myui/hivemall)
 
-**News:** Hivemall joins [Apache Incubator](http://incubator.apache.org/projects/hivemall.html)! :tada: Currently in the process of moving the project repository to ASF.
-[![Incubator](http://incubator.apache.org/images/egg-logo2.png "Apache Incubator")](http://incubator.apache.org/projects/hivemall.html)
+**News:** Hivemall joins [Apache Incubator](http://hivemall.incubator.apache.org/)! :tada: Currently in the process of moving the project repository to ASF.
+[![Incubator](http://incubator.apache.org/images/egg-logo2.png "Apache Incubator")](http://hivemall.incubator.apache.org/)
 
 Hivemall is a scalable machine learning library that runs on Apache Hive.
 Hivemall is designed to be scalable to the number of training instances as well as the number of training features.

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/77d783ad/src/site/site.xml
----------------------------------------------------------------------
diff --git a/src/site/site.xml b/src/site/site.xml
index 8a20d84..3a673ed 100644
--- a/src/site/site.xml
+++ b/src/site/site.xml
@@ -80,6 +80,7 @@
 			<item name="Licenses" href="/license.html" />
 			<item name="Team" href="/team-list.html" />
 			<item name="Powered By" href="/poweredby.html" />
+			<item name="Incubation Status" href="http://incubator.apache.org/projects/hivemall.html" />
 		</menu>
 		
 		<menu name="Documentation">


[13/50] [abbrv] incubator-hivemall git commit: Updated license headers

Posted by my...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/test/java/hivemall/utils/lang/NumberUtilsTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/hivemall/utils/lang/NumberUtilsTest.java b/core/src/test/java/hivemall/utils/lang/NumberUtilsTest.java
index 5c66a24..751c42d 100644
--- a/core/src/test/java/hivemall/utils/lang/NumberUtilsTest.java
+++ b/core/src/test/java/hivemall/utils/lang/NumberUtilsTest.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.utils.lang;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/test/java/hivemall/utils/math/MatrixUtilsTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/hivemall/utils/math/MatrixUtilsTest.java b/core/src/test/java/hivemall/utils/math/MatrixUtilsTest.java
index bc960ec..e122fbf 100644
--- a/core/src/test/java/hivemall/utils/math/MatrixUtilsTest.java
+++ b/core/src/test/java/hivemall/utils/math/MatrixUtilsTest.java
@@ -1,19 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.utils.math;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/mixserv/pom.xml
----------------------------------------------------------------------
diff --git a/mixserv/pom.xml b/mixserv/pom.xml
index 7a4a1b8..3195613 100644
--- a/mixserv/pom.xml
+++ b/mixserv/pom.xml
@@ -13,6 +13,10 @@
 	<name>Hivemall Mix Server</name>
 	<packaging>jar</packaging>
 
+	<properties>
+		<main.basedir>${project.parent.basedir}</main.basedir>
+	</properties>
+
 	<dependencies>
 		<!-- provided scope -->
 		<dependency>
@@ -119,26 +123,6 @@
 		<finalName>${project.artifactId}-${project.version}</finalName>
 		<testOutputDirectory>target/test-classes</testOutputDirectory>
 		<plugins>
-			<plugin>
-				<groupId>com.mycila</groupId>
-				<artifactId>license-maven-plugin</artifactId>
-				<version>2.8</version>
-				<configuration>
-					<header>${project.parent.basedir}/resources/license-header.txt</header>
-					<properties>
-						<currentYear>${build.year}</currentYear>
-						<copyrightOwner>${project.organization.name}</copyrightOwner>
-					</properties>
-					<includes>
-						<include>src/main/**/*.java</include>
-						<include>src/test/**/*.java</include>
-					</includes>
-					<encoding>UTF-8</encoding>
-					<headerDefinitions>
-						<headerDefinition>${project.parent.basedir}/resources/header-definition.xml</headerDefinition>
-					</headerDefinitions>
-				</configuration>
-			</plugin>
 			<!-- hivemall-mixserv-xx-fat.jar including all dependencies -->
 			<plugin>
 				<groupId>org.apache.maven.plugins</groupId>

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/mixserv/src/main/java/hivemall/mix/metrics/MetricsRegistry.java
----------------------------------------------------------------------
diff --git a/mixserv/src/main/java/hivemall/mix/metrics/MetricsRegistry.java b/mixserv/src/main/java/hivemall/mix/metrics/MetricsRegistry.java
index 9452f78..e7fb757 100644
--- a/mixserv/src/main/java/hivemall/mix/metrics/MetricsRegistry.java
+++ b/mixserv/src/main/java/hivemall/mix/metrics/MetricsRegistry.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.mix.metrics;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/mixserv/src/main/java/hivemall/mix/metrics/MixServerMetrics.java
----------------------------------------------------------------------
diff --git a/mixserv/src/main/java/hivemall/mix/metrics/MixServerMetrics.java b/mixserv/src/main/java/hivemall/mix/metrics/MixServerMetrics.java
index 51b9bae..33a25ba 100644
--- a/mixserv/src/main/java/hivemall/mix/metrics/MixServerMetrics.java
+++ b/mixserv/src/main/java/hivemall/mix/metrics/MixServerMetrics.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.mix.metrics;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/mixserv/src/main/java/hivemall/mix/metrics/MixServerMetricsMBean.java
----------------------------------------------------------------------
diff --git a/mixserv/src/main/java/hivemall/mix/metrics/MixServerMetricsMBean.java b/mixserv/src/main/java/hivemall/mix/metrics/MixServerMetricsMBean.java
index 3557679..4a301ce 100644
--- a/mixserv/src/main/java/hivemall/mix/metrics/MixServerMetricsMBean.java
+++ b/mixserv/src/main/java/hivemall/mix/metrics/MixServerMetricsMBean.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.mix.metrics;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/mixserv/src/main/java/hivemall/mix/metrics/ThroughputCounter.java
----------------------------------------------------------------------
diff --git a/mixserv/src/main/java/hivemall/mix/metrics/ThroughputCounter.java b/mixserv/src/main/java/hivemall/mix/metrics/ThroughputCounter.java
index f4030e6..d60cd73 100644
--- a/mixserv/src/main/java/hivemall/mix/metrics/ThroughputCounter.java
+++ b/mixserv/src/main/java/hivemall/mix/metrics/ThroughputCounter.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.mix.metrics;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/mixserv/src/main/java/hivemall/mix/server/MixServer.java
----------------------------------------------------------------------
diff --git a/mixserv/src/main/java/hivemall/mix/server/MixServer.java b/mixserv/src/main/java/hivemall/mix/server/MixServer.java
index f226176..67ed25f 100644
--- a/mixserv/src/main/java/hivemall/mix/server/MixServer.java
+++ b/mixserv/src/main/java/hivemall/mix/server/MixServer.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.mix.server;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/mixserv/src/main/java/hivemall/mix/server/MixServerHandler.java
----------------------------------------------------------------------
diff --git a/mixserv/src/main/java/hivemall/mix/server/MixServerHandler.java b/mixserv/src/main/java/hivemall/mix/server/MixServerHandler.java
index 7f2df7a..ae5c27c 100644
--- a/mixserv/src/main/java/hivemall/mix/server/MixServerHandler.java
+++ b/mixserv/src/main/java/hivemall/mix/server/MixServerHandler.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.mix.server;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/mixserv/src/main/java/hivemall/mix/server/MixServerInitializer.java
----------------------------------------------------------------------
diff --git a/mixserv/src/main/java/hivemall/mix/server/MixServerInitializer.java b/mixserv/src/main/java/hivemall/mix/server/MixServerInitializer.java
index 3f10801..8117ae8 100644
--- a/mixserv/src/main/java/hivemall/mix/server/MixServerInitializer.java
+++ b/mixserv/src/main/java/hivemall/mix/server/MixServerInitializer.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.mix.server;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/mixserv/src/main/java/hivemall/mix/store/PartialArgminKLD.java
----------------------------------------------------------------------
diff --git a/mixserv/src/main/java/hivemall/mix/store/PartialArgminKLD.java b/mixserv/src/main/java/hivemall/mix/store/PartialArgminKLD.java
index d5d105d..a6ffc4b 100644
--- a/mixserv/src/main/java/hivemall/mix/store/PartialArgminKLD.java
+++ b/mixserv/src/main/java/hivemall/mix/store/PartialArgminKLD.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.mix.store;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/mixserv/src/main/java/hivemall/mix/store/PartialAverage.java
----------------------------------------------------------------------
diff --git a/mixserv/src/main/java/hivemall/mix/store/PartialAverage.java b/mixserv/src/main/java/hivemall/mix/store/PartialAverage.java
index f7d1164..e03a6d5 100644
--- a/mixserv/src/main/java/hivemall/mix/store/PartialAverage.java
+++ b/mixserv/src/main/java/hivemall/mix/store/PartialAverage.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.mix.store;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/mixserv/src/main/java/hivemall/mix/store/PartialResult.java
----------------------------------------------------------------------
diff --git a/mixserv/src/main/java/hivemall/mix/store/PartialResult.java b/mixserv/src/main/java/hivemall/mix/store/PartialResult.java
index 61947df..c52d72b 100644
--- a/mixserv/src/main/java/hivemall/mix/store/PartialResult.java
+++ b/mixserv/src/main/java/hivemall/mix/store/PartialResult.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.mix.store;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/mixserv/src/main/java/hivemall/mix/store/SessionObject.java
----------------------------------------------------------------------
diff --git a/mixserv/src/main/java/hivemall/mix/store/SessionObject.java b/mixserv/src/main/java/hivemall/mix/store/SessionObject.java
index fa9e9e3..7e275aa 100644
--- a/mixserv/src/main/java/hivemall/mix/store/SessionObject.java
+++ b/mixserv/src/main/java/hivemall/mix/store/SessionObject.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.mix.store;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/mixserv/src/main/java/hivemall/mix/store/SessionStore.java
----------------------------------------------------------------------
diff --git a/mixserv/src/main/java/hivemall/mix/store/SessionStore.java b/mixserv/src/main/java/hivemall/mix/store/SessionStore.java
index 8453b10..1a2892e 100644
--- a/mixserv/src/main/java/hivemall/mix/store/SessionStore.java
+++ b/mixserv/src/main/java/hivemall/mix/store/SessionStore.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.mix.store;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/mixserv/src/test/java/hivemall/mix/server/MixServerHandlerTest.java
----------------------------------------------------------------------
diff --git a/mixserv/src/test/java/hivemall/mix/server/MixServerHandlerTest.java b/mixserv/src/test/java/hivemall/mix/server/MixServerHandlerTest.java
index 55acb8f..fbb9cc1 100644
--- a/mixserv/src/test/java/hivemall/mix/server/MixServerHandlerTest.java
+++ b/mixserv/src/test/java/hivemall/mix/server/MixServerHandlerTest.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.mix.server;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/mixserv/src/test/java/hivemall/mix/server/MixServerTest.java
----------------------------------------------------------------------
diff --git a/mixserv/src/test/java/hivemall/mix/server/MixServerTest.java b/mixserv/src/test/java/hivemall/mix/server/MixServerTest.java
index 0b1455c..3a65da8 100644
--- a/mixserv/src/test/java/hivemall/mix/server/MixServerTest.java
+++ b/mixserv/src/test/java/hivemall/mix/server/MixServerTest.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.mix.server;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/mixserv/src/test/java/hivemall/mix/server/PartialResultTest.java
----------------------------------------------------------------------
diff --git a/mixserv/src/test/java/hivemall/mix/server/PartialResultTest.java b/mixserv/src/test/java/hivemall/mix/server/PartialResultTest.java
index f0e344a..5e6ecf2 100644
--- a/mixserv/src/test/java/hivemall/mix/server/PartialResultTest.java
+++ b/mixserv/src/test/java/hivemall/mix/server/PartialResultTest.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.mix.server;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/mixserv/src/test/java/hivemall/test/HivemallTestBase.java
----------------------------------------------------------------------
diff --git a/mixserv/src/test/java/hivemall/test/HivemallTestBase.java b/mixserv/src/test/java/hivemall/test/HivemallTestBase.java
index 341db66..ee80eba 100644
--- a/mixserv/src/test/java/hivemall/test/HivemallTestBase.java
+++ b/mixserv/src/test/java/hivemall/test/HivemallTestBase.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.test;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/nlp/pom.xml
----------------------------------------------------------------------
diff --git a/nlp/pom.xml b/nlp/pom.xml
index 0402e62..82a078f 100644
--- a/nlp/pom.xml
+++ b/nlp/pom.xml
@@ -13,6 +13,10 @@
 	<name>Hivemall NLP</name>
 	<packaging>jar</packaging>
 
+	<properties>
+		<main.basedir>${project.parent.basedir}</main.basedir>
+	</properties>
+
 	<dependencies>
 		<!-- provided scope -->
 		<dependency>
@@ -112,26 +116,6 @@
 		<finalName>${project.artifactId}-${project.version}</finalName>
 		<testOutputDirectory>target/test-classes</testOutputDirectory>
 		<plugins>
-			<plugin>
-				<groupId>com.mycila</groupId>
-				<artifactId>license-maven-plugin</artifactId>
-				<version>2.8</version>
-				<configuration>
-					<header>${project.parent.basedir}/resources/license-header.txt</header>
-					<properties>
-						<currentYear>${build.year}</currentYear>
-						<copyrightOwner>${project.organization.name}</copyrightOwner>
-					</properties>
-					<includes>
-						<include>src/main/**/*.java</include>
-						<include>src/test/**/*.java</include>
-					</includes>
-					<encoding>UTF-8</encoding>
-					<headerDefinitions>
-						<headerDefinition>${project.parent.basedir}/resources/header-definition.xml</headerDefinition>
-					</headerDefinitions>
-				</configuration>
-			</plugin>
 			<!-- hivemall-nlp-xx.jar -->
 			<plugin>
 				<groupId>org.apache.maven.plugins</groupId>

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/nlp/src/main/java/hivemall/nlp/tokenizer/KuromojiUDF.java
----------------------------------------------------------------------
diff --git a/nlp/src/main/java/hivemall/nlp/tokenizer/KuromojiUDF.java b/nlp/src/main/java/hivemall/nlp/tokenizer/KuromojiUDF.java
index 425a40f..ea977cc 100644
--- a/nlp/src/main/java/hivemall/nlp/tokenizer/KuromojiUDF.java
+++ b/nlp/src/main/java/hivemall/nlp/tokenizer/KuromojiUDF.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/nlp/src/test/java/hivemall/nlp/tokenizer/KuromojiUDFTest.java
----------------------------------------------------------------------
diff --git a/nlp/src/test/java/hivemall/nlp/tokenizer/KuromojiUDFTest.java b/nlp/src/test/java/hivemall/nlp/tokenizer/KuromojiUDFTest.java
index 4bdde78..005e689 100644
--- a/nlp/src/test/java/hivemall/nlp/tokenizer/KuromojiUDFTest.java
+++ b/nlp/src/test/java/hivemall/nlp/tokenizer/KuromojiUDFTest.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 82496cb..dcc0917 100644
--- a/pom.xml
+++ b/pom.xml
@@ -220,6 +220,7 @@
 		<scala.version>2.11.8</scala.version>
 		<dependency.locations.enabled>false</dependency.locations.enabled>
 		<scala.binary.version>2.11</scala.binary.version>
+		<main.basedir>${project.basedir}</main.basedir>
 	</properties>
 
 	<repositories>
@@ -369,15 +370,46 @@
 		<pluginManagement>
 			<plugins>
 				<plugin>
+				    <!-- mvn formatter:format -->
 					<groupId>net.revelc.code</groupId>
 					<artifactId>formatter-maven-plugin</artifactId>
 					<version>0.5.2</version>
 					<configuration>
-						<configFile>${project.parent.basedir}/resources/eclipse-style.xml</configFile>
+						<configFile>${main.basedir}/resources/eclipse-style.xml</configFile>
 						<encoding>UTF-8</encoding>
 						<lineEnding>LF</lineEnding>
 					</configuration>
 				</plugin>
+				<plugin>
+				    <!-- mvn license:check; mvn license:format -->
+					<groupId>com.mycila</groupId>
+					<artifactId>license-maven-plugin</artifactId>
+					<version>3.0</version>
+					<configuration>
+						<header>${main.basedir}/resources/license-header.txt</header>
+						<properties>
+							<currentYear>${build.year}</currentYear>
+							<copyrightOwner>${project.organization.name}</copyrightOwner>
+						</properties>
+						<includes>
+							<include>src/main/**/*.java</include>
+							<include>src/test/**/*.java</include>
+							<include>src/main/**/*.scala</include>
+							<include>src/test/**/*.scala</include>
+						</includes>
+						<encoding>UTF-8</encoding>
+						<headerDefinitions>
+							<headerDefinition>${main.basedir}/resources/header-definition.xml</headerDefinition>
+						</headerDefinitions>
+					</configuration>
+					<executions>
+						<execution>
+							<goals>
+								<goal>check</goal>
+							</goals>
+						</execution>
+					</executions>
+				</plugin>
 			</plugins>
 		</pluginManagement>
 
@@ -388,6 +420,11 @@
 				<artifactId>formatter-maven-plugin</artifactId>
 			</plugin>
 			<plugin>
+			    <!-- mvn license:check; mvn license:format -->
+				<groupId>com.mycila</groupId>
+				<artifactId>license-maven-plugin</artifactId>
+			</plugin>
+			<plugin>
 				<groupId>org.apache.maven.plugins</groupId>
 				<artifactId>maven-failsafe-plugin</artifactId>
 				<version>2.17</version>

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/resources/license-header.txt
----------------------------------------------------------------------
diff --git a/resources/license-header.txt b/resources/license-header.txt
index a7fb17c..90705e0 100644
--- a/resources/license-header.txt
+++ b/resources/license-header.txt
@@ -1,16 +1,16 @@
-Hivemall: Hive scalable Machine Learning Library
+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
 
-Copyright (C) ${currentYear} ${copyrightOwner}
-Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+  http://www.apache.org/licenses/LICENSE-2.0
 
-Licensed 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.
\ No newline at end of file
+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.
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/spark/spark-1.6/extra-src/hive/src/main/scala/org/apache/spark/sql/hive/HiveShim.scala
----------------------------------------------------------------------
diff --git a/spark/spark-1.6/extra-src/hive/src/main/scala/org/apache/spark/sql/hive/HiveShim.scala b/spark/spark-1.6/extra-src/hive/src/main/scala/org/apache/spark/sql/hive/HiveShim.scala
index ca636b0..6da54f7 100644
--- a/spark/spark-1.6/extra-src/hive/src/main/scala/org/apache/spark/sql/hive/HiveShim.scala
+++ b/spark/spark-1.6/extra-src/hive/src/main/scala/org/apache/spark/sql/hive/HiveShim.scala
@@ -1,20 +1,21 @@
 /*
- * 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
+ * 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
+ *   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.
+ * 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 org.apache.spark.sql.hive.client
 
 import java.lang.{Boolean => JBoolean, Integer => JInteger, Long => JLong}

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/spark/spark-1.6/pom.xml
----------------------------------------------------------------------
diff --git a/spark/spark-1.6/pom.xml b/spark/spark-1.6/pom.xml
index fd53baf..99c4a5e 100644
--- a/spark/spark-1.6/pom.xml
+++ b/spark/spark-1.6/pom.xml
@@ -17,6 +17,7 @@
 		<PermGen>64m</PermGen>
 		<MaxPermGen>1024m</MaxPermGen>
 		<CodeCacheSize>512m</CodeCacheSize>
+		<main.basedir>${project.parent.basedir}</main.basedir>
 	</properties>
 
 	<dependencies>

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/spark/spark-1.6/src/main/scala/hivemall/tools/RegressionDatagen.scala
----------------------------------------------------------------------
diff --git a/spark/spark-1.6/src/main/scala/hivemall/tools/RegressionDatagen.scala b/spark/spark-1.6/src/main/scala/hivemall/tools/RegressionDatagen.scala
index 9b90b98..01664f4 100644
--- a/spark/spark-1.6/src/main/scala/hivemall/tools/RegressionDatagen.scala
+++ b/spark/spark-1.6/src/main/scala/hivemall/tools/RegressionDatagen.scala
@@ -1,20 +1,21 @@
 /*
- * 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
+ * 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
+ *   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.
+ * 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.tools
 
 import org.apache.spark.sql.{DataFrame, Row, SQLContext}

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/spark/spark-1.6/src/main/scala/org/apache/spark/sql/hive/GroupedDataEx.scala
----------------------------------------------------------------------
diff --git a/spark/spark-1.6/src/main/scala/org/apache/spark/sql/hive/GroupedDataEx.scala b/spark/spark-1.6/src/main/scala/org/apache/spark/sql/hive/GroupedDataEx.scala
index a87567e..fd4da64 100644
--- a/spark/spark-1.6/src/main/scala/org/apache/spark/sql/hive/GroupedDataEx.scala
+++ b/spark/spark-1.6/src/main/scala/org/apache/spark/sql/hive/GroupedDataEx.scala
@@ -1,20 +1,21 @@
 /*
- * 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
+ * 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
+ *   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.
+ * 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 org.apache.spark.sql.hive
 
 import org.apache.spark.sql.{AnalysisException, DataFrame, GroupedData}

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/spark/spark-1.6/src/main/scala/org/apache/spark/sql/hive/HivemallOps.scala
----------------------------------------------------------------------
diff --git a/spark/spark-1.6/src/main/scala/org/apache/spark/sql/hive/HivemallOps.scala b/spark/spark-1.6/src/main/scala/org/apache/spark/sql/hive/HivemallOps.scala
index eea4d9a..4c750dd 100644
--- a/spark/spark-1.6/src/main/scala/org/apache/spark/sql/hive/HivemallOps.scala
+++ b/spark/spark-1.6/src/main/scala/org/apache/spark/sql/hive/HivemallOps.scala
@@ -1,20 +1,21 @@
 /*
- * 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
+ * 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
+ *   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.
+ * 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 org.apache.spark.sql.hive
 
 import java.util.UUID

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/spark/spark-1.6/src/main/scala/org/apache/spark/sql/hive/HivemallUtils.scala
----------------------------------------------------------------------
diff --git a/spark/spark-1.6/src/main/scala/org/apache/spark/sql/hive/HivemallUtils.scala b/spark/spark-1.6/src/main/scala/org/apache/spark/sql/hive/HivemallUtils.scala
index bccecec..dff62b3 100644
--- a/spark/spark-1.6/src/main/scala/org/apache/spark/sql/hive/HivemallUtils.scala
+++ b/spark/spark-1.6/src/main/scala/org/apache/spark/sql/hive/HivemallUtils.scala
@@ -1,20 +1,21 @@
 /*
- * 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
+ * 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
+ *   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.
+ * 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 org.apache.spark.sql.hive
 
 import org.apache.spark.mllib.linalg.{BLAS, Vector, Vectors}

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/spark/spark-1.6/src/test/scala/hivemall/mix/server/MixServerSuite.scala
----------------------------------------------------------------------
diff --git a/spark/spark-1.6/src/test/scala/hivemall/mix/server/MixServerSuite.scala b/spark/spark-1.6/src/test/scala/hivemall/mix/server/MixServerSuite.scala
index cb3abee..dbb818b 100644
--- a/spark/spark-1.6/src/test/scala/hivemall/mix/server/MixServerSuite.scala
+++ b/spark/spark-1.6/src/test/scala/hivemall/mix/server/MixServerSuite.scala
@@ -1,20 +1,21 @@
 /*
- * 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
+ * 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
+ *   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.
+ * 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.mix.server
 
 import java.util.Random

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/spark/spark-1.6/src/test/scala/hivemall/tools/RegressionDatagenSuite.scala
----------------------------------------------------------------------
diff --git a/spark/spark-1.6/src/test/scala/hivemall/tools/RegressionDatagenSuite.scala b/spark/spark-1.6/src/test/scala/hivemall/tools/RegressionDatagenSuite.scala
index 90c955c..f203fc2 100644
--- a/spark/spark-1.6/src/test/scala/hivemall/tools/RegressionDatagenSuite.scala
+++ b/spark/spark-1.6/src/test/scala/hivemall/tools/RegressionDatagenSuite.scala
@@ -1,20 +1,21 @@
 /*
- * 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
+ * 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
+ *   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.
+ * 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.tools
 
 import org.scalatest.FunSuite

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/spark/spark-1.6/src/test/scala/org/apache/spark/SparkFunSuite.scala
----------------------------------------------------------------------
diff --git a/spark/spark-1.6/src/test/scala/org/apache/spark/SparkFunSuite.scala b/spark/spark-1.6/src/test/scala/org/apache/spark/SparkFunSuite.scala
index 8b065f5..991e46f 100644
--- a/spark/spark-1.6/src/test/scala/org/apache/spark/SparkFunSuite.scala
+++ b/spark/spark-1.6/src/test/scala/org/apache/spark/SparkFunSuite.scala
@@ -1,20 +1,21 @@
 /*
- * 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
+ * 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
+ *   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.
+ * 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 org.apache.spark
 
 // scalastyle:off

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/spark/spark-1.6/src/test/scala/org/apache/spark/ml/feature/HivemallLabeledPointSuite.scala
----------------------------------------------------------------------
diff --git a/spark/spark-1.6/src/test/scala/org/apache/spark/ml/feature/HivemallLabeledPointSuite.scala b/spark/spark-1.6/src/test/scala/org/apache/spark/ml/feature/HivemallLabeledPointSuite.scala
index f9d207e..f57983f 100644
--- a/spark/spark-1.6/src/test/scala/org/apache/spark/ml/feature/HivemallLabeledPointSuite.scala
+++ b/spark/spark-1.6/src/test/scala/org/apache/spark/ml/feature/HivemallLabeledPointSuite.scala
@@ -1,20 +1,21 @@
 /*
- * 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
+ * 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
+ *   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.
+ * 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 org.apache.spark.ml.feature
 
 import org.apache.spark.SparkFunSuite

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/spark/spark-1.6/src/test/scala/org/apache/spark/sql/QueryTest.scala
----------------------------------------------------------------------
diff --git a/spark/spark-1.6/src/test/scala/org/apache/spark/sql/QueryTest.scala b/spark/spark-1.6/src/test/scala/org/apache/spark/sql/QueryTest.scala
index bd21d46..ef520ae 100644
--- a/spark/spark-1.6/src/test/scala/org/apache/spark/sql/QueryTest.scala
+++ b/spark/spark-1.6/src/test/scala/org/apache/spark/sql/QueryTest.scala
@@ -1,20 +1,21 @@
 /*
- * 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
+ * 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
+ *   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.
+ * 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 org.apache.spark.sql
 
 import java.util.{Locale, TimeZone}

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/spark/spark-1.6/src/test/scala/org/apache/spark/sql/catalyst/plans/PlanTest.scala
----------------------------------------------------------------------
diff --git a/spark/spark-1.6/src/test/scala/org/apache/spark/sql/catalyst/plans/PlanTest.scala b/spark/spark-1.6/src/test/scala/org/apache/spark/sql/catalyst/plans/PlanTest.scala
index 754d0bc..816576e 100644
--- a/spark/spark-1.6/src/test/scala/org/apache/spark/sql/catalyst/plans/PlanTest.scala
+++ b/spark/spark-1.6/src/test/scala/org/apache/spark/sql/catalyst/plans/PlanTest.scala
@@ -1,20 +1,21 @@
 /*
- * 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
+ * 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
+ *   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.
+ * 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 org.apache.spark.sql.catalyst.plans
 
 import org.apache.spark.SparkFunSuite

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/spark/spark-1.6/src/test/scala/org/apache/spark/sql/hive/HiveUdfSuite.scala
----------------------------------------------------------------------
diff --git a/spark/spark-1.6/src/test/scala/org/apache/spark/sql/hive/HiveUdfSuite.scala b/spark/spark-1.6/src/test/scala/org/apache/spark/sql/hive/HiveUdfSuite.scala
index 61f35d0..ded94ba 100644
--- a/spark/spark-1.6/src/test/scala/org/apache/spark/sql/hive/HiveUdfSuite.scala
+++ b/spark/spark-1.6/src/test/scala/org/apache/spark/sql/hive/HiveUdfSuite.scala
@@ -1,20 +1,21 @@
 /*
- * 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
+ * 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
+ *   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.
+ * 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 org.apache.spark.sql.hive
 
 import org.apache.spark.sql.Row

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/spark/spark-1.6/src/test/scala/org/apache/spark/sql/hive/HivemallOpsSuite.scala
----------------------------------------------------------------------
diff --git a/spark/spark-1.6/src/test/scala/org/apache/spark/sql/hive/HivemallOpsSuite.scala b/spark/spark-1.6/src/test/scala/org/apache/spark/sql/hive/HivemallOpsSuite.scala
index 25d900e..901056d 100644
--- a/spark/spark-1.6/src/test/scala/org/apache/spark/sql/hive/HivemallOpsSuite.scala
+++ b/spark/spark-1.6/src/test/scala/org/apache/spark/sql/hive/HivemallOpsSuite.scala
@@ -1,20 +1,21 @@
 /*
- * 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
+ * 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
+ *   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.
+ * 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 org.apache.spark.sql.hive
 
 import org.apache.spark.sql.{Column, Row}

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/spark/spark-1.6/src/test/scala/org/apache/spark/sql/hive/ModelMixingSuite.scala
----------------------------------------------------------------------
diff --git a/spark/spark-1.6/src/test/scala/org/apache/spark/sql/hive/ModelMixingSuite.scala b/spark/spark-1.6/src/test/scala/org/apache/spark/sql/hive/ModelMixingSuite.scala
index 423549e..276bc13 100644
--- a/spark/spark-1.6/src/test/scala/org/apache/spark/sql/hive/ModelMixingSuite.scala
+++ b/spark/spark-1.6/src/test/scala/org/apache/spark/sql/hive/ModelMixingSuite.scala
@@ -1,20 +1,21 @@
 /*
- * 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
+ * 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
+ *   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.
+ * 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 org.apache.spark.sql.hive
 
 import java.io.{BufferedInputStream, BufferedReader, InputStream, InputStreamReader}

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/spark/spark-1.6/src/test/scala/org/apache/spark/streaming/HivemallOpsSuite.scala
----------------------------------------------------------------------
diff --git a/spark/spark-1.6/src/test/scala/org/apache/spark/streaming/HivemallOpsSuite.scala b/spark/spark-1.6/src/test/scala/org/apache/spark/streaming/HivemallOpsSuite.scala
index 42ffc03..5cc590a 100644
--- a/spark/spark-1.6/src/test/scala/org/apache/spark/streaming/HivemallOpsSuite.scala
+++ b/spark/spark-1.6/src/test/scala/org/apache/spark/streaming/HivemallOpsSuite.scala
@@ -1,20 +1,21 @@
 /*
- * 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
+ * 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
+ *   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.
+ * 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 org.apache.spark.streaming
 
 import reflect.ClassTag

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/spark/spark-1.6/src/test/scala/org/apache/spark/test/HivemallQueryTest.scala
----------------------------------------------------------------------
diff --git a/spark/spark-1.6/src/test/scala/org/apache/spark/test/HivemallQueryTest.scala b/spark/spark-1.6/src/test/scala/org/apache/spark/test/HivemallQueryTest.scala
index 2bb8a0b..020cf8c 100644
--- a/spark/spark-1.6/src/test/scala/org/apache/spark/test/HivemallQueryTest.scala
+++ b/spark/spark-1.6/src/test/scala/org/apache/spark/test/HivemallQueryTest.scala
@@ -1,20 +1,21 @@
 /*
- * 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
+ * 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
+ *   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.
+ * 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 org.apache.spark.test
 
 import scala.collection.mutable.Seq


[29/50] [abbrv] incubator-hivemall git commit: Updated license headers of scalastyle-config and pom.xml

Posted by my...@apache.org.
Updated license headers of scalastyle-config and pom.xml

Project: http://git-wip-us.apache.org/repos/asf/incubator-hivemall/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-hivemall/commit/67bdcb81
Tree: http://git-wip-us.apache.org/repos/asf/incubator-hivemall/tree/67bdcb81
Diff: http://git-wip-us.apache.org/repos/asf/incubator-hivemall/diff/67bdcb81

Branch: refs/heads/master
Commit: 67bdcb811fcdf8b2e0ddc0e0b6dc3a015f00c05d
Parents: 02b839b
Author: myui <yu...@gmail.com>
Authored: Mon Nov 7 15:39:07 2016 +0900
Committer: myui <yu...@gmail.com>
Committed: Mon Nov 7 15:39:07 2016 +0900

----------------------------------------------------------------------
 core/pom.xml                             | 18 ++++++++
 mixserv/pom.xml                          | 18 ++++++++
 nlp/pom.xml                              | 18 ++++++++
 pom.xml                                  | 19 +++++++++
 spark/spark-1.6/pom.xml                  | 18 ++++++++
 spark/spark-2.0/pom.xml                  | 18 ++++++++
 spark/spark-common/pom.xml               | 18 ++++++++
 spark/spark-common/scalastyle-config.xml | 60 ++++++++++++++-------------
 xgboost/pom.xml                          | 18 ++++++++
 9 files changed, 177 insertions(+), 28 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/67bdcb81/core/pom.xml
----------------------------------------------------------------------
diff --git a/core/pom.xml b/core/pom.xml
index bddca2a..bf931ac 100644
--- a/core/pom.xml
+++ b/core/pom.xml
@@ -1,3 +1,21 @@
+<!--
+  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.
+-->
 <project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
 	xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
 	<modelVersion>4.0.0</modelVersion>

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/67bdcb81/mixserv/pom.xml
----------------------------------------------------------------------
diff --git a/mixserv/pom.xml b/mixserv/pom.xml
index 3195613..41ba401 100644
--- a/mixserv/pom.xml
+++ b/mixserv/pom.xml
@@ -1,3 +1,21 @@
+<!--
+  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.
+-->
 <project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
 	xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
 	<modelVersion>4.0.0</modelVersion>

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/67bdcb81/nlp/pom.xml
----------------------------------------------------------------------
diff --git a/nlp/pom.xml b/nlp/pom.xml
index 82a078f..c7d2cef 100644
--- a/nlp/pom.xml
+++ b/nlp/pom.xml
@@ -1,3 +1,21 @@
+<!--
+  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.
+-->
 <project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
 	xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
 	<modelVersion>4.0.0</modelVersion>

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/67bdcb81/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 094f034..21a9a57 100644
--- a/pom.xml
+++ b/pom.xml
@@ -1,3 +1,21 @@
+<!--
+  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.
+-->
 <project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
 	xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
 	<modelVersion>4.0.0</modelVersion>
@@ -398,6 +416,7 @@
 							<include>src/test/**/*.scala</include>
 							<include>bin/*.sh</include>
 							<include>docs/gitbook/**/*.md</include>
+							<include>**/pom.xml</include>
 						</includes>
 						<useDefaultExcludes>false</useDefaultExcludes>
 						<excludes>

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/67bdcb81/spark/spark-1.6/pom.xml
----------------------------------------------------------------------
diff --git a/spark/spark-1.6/pom.xml b/spark/spark-1.6/pom.xml
index 99c4a5e..2170ed6 100644
--- a/spark/spark-1.6/pom.xml
+++ b/spark/spark-1.6/pom.xml
@@ -1,3 +1,21 @@
+<!--
+  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.
+-->
 <project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
 	xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
 	<modelVersion>4.0.0</modelVersion>

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/67bdcb81/spark/spark-2.0/pom.xml
----------------------------------------------------------------------
diff --git a/spark/spark-2.0/pom.xml b/spark/spark-2.0/pom.xml
index 79a43de..95ffff2 100644
--- a/spark/spark-2.0/pom.xml
+++ b/spark/spark-2.0/pom.xml
@@ -1,3 +1,21 @@
+<!--
+  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.
+-->
 <project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
 	xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
 	<modelVersion>4.0.0</modelVersion>

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/67bdcb81/spark/spark-common/pom.xml
----------------------------------------------------------------------
diff --git a/spark/spark-common/pom.xml b/spark/spark-common/pom.xml
index 6325579..7d76ea4 100644
--- a/spark/spark-common/pom.xml
+++ b/spark/spark-common/pom.xml
@@ -1,3 +1,21 @@
+<!--
+  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.
+-->
 <project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
 	xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
 	<modelVersion>4.0.0</modelVersion>

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/67bdcb81/spark/spark-common/scalastyle-config.xml
----------------------------------------------------------------------
diff --git a/spark/spark-common/scalastyle-config.xml b/spark/spark-common/scalastyle-config.xml
index 06ff10f..13d1c47 100644
--- a/spark/spark-common/scalastyle-config.xml
+++ b/spark/spark-common/scalastyle-config.xml
@@ -1,21 +1,23 @@
 <!--
-  ~ 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.
-  -->
-<!--
+  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.
+-->
 
+<!--
 If you wish to turn off checking for a section of code, you can put a comment in the source
 before and after the section, with the following syntax:
 
@@ -49,20 +51,22 @@ This file is divided into 3 sections:
   <check level="error" class="org.scalastyle.file.HeaderMatchesChecker" enabled="true">
     <parameters>
        <parameter name="header"><![CDATA[/*
- * 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
+ * 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
+ *   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.
+ * 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.
  */]]></parameter>
     </parameters>
   </check>

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/67bdcb81/xgboost/pom.xml
----------------------------------------------------------------------
diff --git a/xgboost/pom.xml b/xgboost/pom.xml
index 677fbbd..cd5dc0d 100644
--- a/xgboost/pom.xml
+++ b/xgboost/pom.xml
@@ -1,3 +1,21 @@
+<!--
+  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.
+-->
 <project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
 	xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
 	<modelVersion>4.0.0</modelVersion>


[46/50] [abbrv] incubator-hivemall git commit: Remove spark tutorials for now (#383)

Posted by my...@apache.org.
Remove spark tutorials for now (#383)



Project: http://git-wip-us.apache.org/repos/asf/incubator-hivemall/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-hivemall/commit/f9f9799d
Tree: http://git-wip-us.apache.org/repos/asf/incubator-hivemall/tree/f9f9799d
Diff: http://git-wip-us.apache.org/repos/asf/incubator-hivemall/diff/f9f9799d

Branch: refs/heads/master
Commit: f9f9799d7b3499624d3c231e71d5dab2eb97b447
Parents: da55b54
Author: Takeshi Yamamuro <li...@gmail.com>
Authored: Mon Nov 21 13:52:09 2016 +0900
Committer: Makoto YUI <yu...@gmail.com>
Committed: Mon Nov 21 13:52:09 2016 +0900

----------------------------------------------------------------------
 .../tutorials/binary_classification.md          | 71 ----------------
 .../tutorials/multiclass_classification.md      | 77 ------------------
 spark/spark-1.6/tutorials/randomforest.md       | 86 --------------------
 spark/spark-1.6/tutorials/regression.md         | 71 ----------------
 .../tutorials/binary_classification.md          | 71 ----------------
 .../tutorials/multiclass_classification.md      | 77 ------------------
 spark/spark-2.0/tutorials/randomforest.md       | 86 --------------------
 spark/spark-2.0/tutorials/regression.md         | 71 ----------------
 8 files changed, 610 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/f9f9799d/spark/spark-1.6/tutorials/binary_classification.md
----------------------------------------------------------------------
diff --git a/spark/spark-1.6/tutorials/binary_classification.md b/spark/spark-1.6/tutorials/binary_classification.md
deleted file mode 100644
index 22b1c29..0000000
--- a/spark/spark-1.6/tutorials/binary_classification.md
+++ /dev/null
@@ -1,71 +0,0 @@
-This tutorial uses [9a binary classification](https://github.com/myui/hivemall/wiki#a9a-binary-classification) as a reference.
-
-Data preparation
---------------------
-```
-// Fetch training data
-# wget http://www.csie.ntu.edu.tw/~cjlin/libsvmtools/datasets/binary/a9a
-# wget http://www.csie.ntu.edu.tw/~cjlin/libsvmtools/datasets/binary/a9a.t
-
-// Fetch a script to normalize the data
-# wget https://raw.githubusercontent.com/maropu/hivemall-spark/master/scripts/misc/conv.awk
-# awk -f conv.awk a9a | sed -e "s/+1/1/" | sed -e "s/-1/0/" > a9a.train
-# awk -f conv.awk a9a.t | sed -e "s/+1/1/" | sed -e "s/-1/0/" > a9a.test
-
-// Fetch an initialization script for hivemall-spark
-# wget https://raw.githubusercontent.com/maropu/hivemall-spark/master/scripts/ddl/define-dfs.sh
-
-// Invoke a spark-shell with hivemall-spark
-# bin/spark-shell --packages maropu:hivemall-spark:0.0.6 --master=local-cluster[2,1,1024] --conf spark.executor.memory=1024
-
-scala> :load define-dfs.sh
-
-scala> :paste
-
-// Load the training data as a RDD
-val trainRdd = sc.textFile("a9a.train")
-  .map(HmLabeledPoint.parse)
-
-// Create the DataFrame that has exactly 2 partitions and
-// amplify the data by 3 times.
-val trainDf = sqlContext.createDataFrame(trainRdd)
-  .coalesce(2).part_amplify(3)
-
-// Load the test data as a RDD
-val testRdd = sc.textFile("a9a.test")
-  .map(HmLabeledPoint.parse)
-
-// Transform into a DataFrame and transform features
-// into a Spark Vector type.
-val testDf = sqlContext.createDataFrame(testRdd)
-  .select($"label".as("target"), ft2vec($"features").as("features"))
-```
-
-Training (Logistic Regression)
---------------------
-```
-// Make a model from the training data
-val model = trainDf
-  .train_logregr(add_bias($"features"), $"label", "-total_steps 32561")
-  .groupby("feature").agg("weight" -> "avg")
-  .as("feature", "weight")
-
-val modelUdf = HivemallUtils
-  .funcModel(model)
-```
-
-Test
---------------------
-```
-// Do prediction
-val predict = testDf
-  .select($"target", sigmoid(modelUdf($"features")).as("prob"))
-  .select($"target", when($"prob" > 0.50, 1.0).otherwise(0.0).as("predict"), $"prob")
-  .cache
-```
-
-Evaluation
---------------------
-```
-(predict.where($"target" === $"predict").count + 0.0) / predict.count
-```

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/f9f9799d/spark/spark-1.6/tutorials/multiclass_classification.md
----------------------------------------------------------------------
diff --git a/spark/spark-1.6/tutorials/multiclass_classification.md b/spark/spark-1.6/tutorials/multiclass_classification.md
deleted file mode 100644
index 74cd3bc..0000000
--- a/spark/spark-1.6/tutorials/multiclass_classification.md
+++ /dev/null
@@ -1,77 +0,0 @@
-This tutorial uses [news20 multiclass classification](https://github.com/myui/hivemall/wiki#news20-multiclass-classification) as a reference.
-
-Data preparation
---------------------
-```
-// Fetch training and test data
-# wget http://www.csie.ntu.edu.tw/~cjlin/libsvmtools/datasets/multiclass/news20.scale.bz2
-# wget http://www.csie.ntu.edu.tw/~cjlin/libsvmtools/datasets/multiclass/news20.t.scale.bz2
-
-// Fetch a script to normalize the data
-# wget https://raw.githubusercontent.com/maropu/hivemall-spark/master/scripts/misc/conv.awk
-# bunzip2 -c news20.scale.bz2 | awk -f conv.awk > news20.train
-# bunzip2 -c news20.t.scale.bz2 | awk -f conv.awk > news20.test
-
-// Fetch an initialization script for hivemall-spark
-# wget https://raw.githubusercontent.com/maropu/hivemall-spark/master/scripts/ddl/define-dfs.sh
-
-// Invoke a spark-shell with hivemall-spark
-# bin/spark-shell --packages maropu:hivemall-spark:0.0.6 --master=local-cluster[2,1,1024] --conf spark.executor.memory=1024
-
-scala> :load define-dfs.sh
-
-scala> :paste
-
-// Load the training data as a RDD
-val trainRdd = sc.textFile("news20.train")
-  .map(HmLabeledPoint.parse)
-
-// Create the DataFrame that has exactly 2 partitions and
-// amplify the data by 3 times.
-val trainDf = sqlContext.createDataFrame(trainRdd)
-  .coalesce(2).part_amplify(3)
-
-// Load the test data as a RDD
-val testRdd = sc.textFile("news20.test")
-  .map(HmLabeledPoint.parse)
-
-// Transform into a DataFrame and transform features
-// into a Spark Vector type.
-val testDf = sqlContext.createDataFrame(testRdd)
-  .select(rowid(), $"label".cast(IntegerType).as("target"), $"features")
-  .cache
-
-val testDf_exploded = testDf.explode_array($"features")
-  .select($"rowid", $"target", extract_feature($"feature"), extract_weight($"feature"))
-```
-
-Training (CW)
---------------------
-```
-// Make a model from the training data
-val model = trainDf
-  .train_multiclass_cw(add_bias($"features"), $"label".cast(IntegerType))
-  .groupby("label", "feature").argmin_kld("weight", "conv")
-  .as("label", "feature", "weight")
-```
-
-Test
---------------------
-```
-// Do prediction
-val predict = testDf_exploded
-  .join(model, testDf_exploded("feature") === model("feature"), "LEFT_OUTER")
-  .select($"rowid", $"label".cast(StringType).as("label"), ($"weight" * $"value").as("value"))
-  .groupby("rowid", "label").sum("value")
-  .groupby("rowid").maxrow("SUM(value)", "label")
-  .as("rowid", "r")
-  .select($"rowid", $"r.col0".as("score"), $"r.col1".as("predict"))
-  .cache
-```
-
-Evaluation
---------------------
-```
-val joinPredicate = (testDf("rowid") === predict("rowid")).and(testDf("target") === predict("predict"))
-(testDf.join(predict, joinPredicate, "INNER").count + 0.0) / testDf.count
-```

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/f9f9799d/spark/spark-1.6/tutorials/randomforest.md
----------------------------------------------------------------------
diff --git a/spark/spark-1.6/tutorials/randomforest.md b/spark/spark-1.6/tutorials/randomforest.md
deleted file mode 100644
index 350e19a..0000000
--- a/spark/spark-1.6/tutorials/randomforest.md
+++ /dev/null
@@ -1,86 +0,0 @@
-This tutorial uses [Kaggle Titanic binary classification](https://github.com/myui/hivemall/wiki/Kaggle-Titanic-binary-classification-using-Random-Forest) as a reference.
-
-Data preparation
---------------------
-```
-// Fetch training and test data in Kaggle(https://www.kaggle.com/c/titanic/data), train.csv and test.csv
-
-// Fetch an initialization script for hivemall-spark
-# wget https://raw.githubusercontent.com/maropu/hivemall-spark/master/scripts/ddl/define-dfs.sh
-
-// Invoke a spark-shell with spark-csv and hivemall-spark
-# bin/spark-shell --packages com.databricks:spark-csv_2.10:1.4.0,maropu:hivemall-spark:0.0.6
-
-scala> :load define-dfs.sh
-
-scala> :paste
-
-// Load the training data as a DataFrame
-val trainCsvDf = sqlContext
-  .read
-  .format("com.databricks.spark.csv")
-  .option("header", "true")
-  .option("inferSchema", "true")
-  .load("train.csv")
-  .cache // Cached for second use
-
-val trainQuantifiedDf = trainCsvDf
-  .quantify(true.as("output") +: trainCsvDf.cols: _*)
-  // Rename output columns for readability
-  .as("passengerid", "survived", "pclass", "name", "sex", "age", "sibsp", "parch", "ticket", "fare", "cabin", "embarked")
-  .sort($"passengerid".asc)
-
-val trainDf = trainQuantifiedDf
-  .select(
-      $"passengerid",
-      array(trainQuantifiedDf.cols.drop(2): _*).as("features"),
-      $"survived"
-    )
-
-// Load the test data as a DataFrame
-val testCsvDf = sqlContext
-  .read
-  .format("com.databricks.spark.csv")
-  .option("header", "true")
-  .option("inferSchema", "true")
-  .load("test.csv")
-
-val testQuantifiedDf = testCsvDf
-  .select(Seq(1.as("train_first"), true.as("output"), $"PassengerId") ++ testCsvDf.cols.drop(1): _*)
-  .unionAll(
-      trainCsvDf.select(Seq(0.as("train_first"), false.as("output"), $"PassengerId") ++ trainCsvDf.cols.drop(2): _*)
-    )
-  .sort($"train_first".asc, $"PassengerId".asc)
-  .quantify($"output" +: testCsvDf.cols: _*)
-  // Rename output columns for readability
-  .as("passengerid", "pclass", "name", "sex", "age", "sibsp", "parch", "ticket", "fare", "cabin", "embarked")
-
-val testDf = testQuantifiedDf
-  .select($"passengerid", array(testQuantifiedDf.cols.drop(1): _*).as("features"))
-```
-
-Training
---------------------
-```
-// Make a model from the training data
-val model = trainDf
-  .coalesce(4)
-  .train_randomforest_classifier($"features", $"survived", "-trees 400")
-```
-
-Test
---------------------
-```
-// Do prediction
-model
-  .coalesce(4)
-  .join(testDf)
-  .select(
-      testDf("passengerid"),
-      tree_predict(model("model_id"), model("model_type"), model("pred_model"), testDf("features"), true).as("predicted")
-    )
-  .groupby($"passengerid").rf_ensemble("predicted")
-  .as("passengerid", "predicted")
-  .select($"passengerid", $"predicted.label")
-  .show
-```

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/f9f9799d/spark/spark-1.6/tutorials/regression.md
----------------------------------------------------------------------
diff --git a/spark/spark-1.6/tutorials/regression.md b/spark/spark-1.6/tutorials/regression.md
deleted file mode 100644
index 5856373..0000000
--- a/spark/spark-1.6/tutorials/regression.md
+++ /dev/null
@@ -1,71 +0,0 @@
-This tutorial uses [E2006 tfidf regression](https://github.com/myui/hivemall/wiki#e2006-tfidf-regression) as a reference.
-
-Data preparation
---------------------
-```
-// Fetch training and test data
-# wget http://www.csie.ntu.edu.tw/~cjlin/libsvmtools/datasets/regression/E2006.train.bz2
-# wget http://www.csie.ntu.edu.tw/~cjlin/libsvmtools/datasets/regression/E2006.test.bz2
-
-// Fetch a script to normalize the data
-# wget https://raw.githubusercontent.com/maropu/hivemall-spark/master/scripts/misc/conv.awk
-# bunzip2 -c E2006.train.bz2 | awk -f conv.awk > E2006.train.lp
-# bunzip2 -c E2006.test.bz2 | awk -f conv.awk > E2006.test.lp
-
-// Fetch an initialization script for hivemall-spark
-# wget https://raw.githubusercontent.com/maropu/hivemall-spark/master/scripts/ddl/define-dfs.sh
-
-// Invoke a spark-shell with hivemall-spark
-# bin/spark-shell --packages maropu:hivemall-spark:0.0.6 --master=local-cluster[2,1,1024] --conf spark.executor.memory=1024
-
-scala> :load define-dfs.sh
-
-scala> :paste
-
-// Load the training data as a RDD
-val trainRdd = sc.textFile("E2006.train.lp")
-  .map(HmLabeledPoint.parse)
-
-// Create the DataFrame that has exactly 2 partitions and
-// amplify the data by 3 times.
-val trainDf = sqlContext.createDataFrame(trainRdd)
-  .coalesce(2).part_amplify(3)
-
-// Load the test data as a RDD
-val testRdd = sc.textFile("E2006.test.lp")
-  .map(HmLabeledPoint.parse)
-
-// Transform into a DataFrame and transform features
-// into a Spark Vector type.
-val testDf = sqlContext.createDataFrame(testRdd)
-  .select($"label".as("target"), ft2vec($"features").as("features"))
-```
-
-Training (PA1)
---------------------
-```
-// Make a model from the training data
-val model = trainDf
-  .train_pa1_regr(add_bias($"features"), $"label")
-  .groupby("feature").agg("weight" -> "avg")
-  .as("feature", "weight")
-
-val modelUdf = HivemallUtils
-  .funcModel(model)
-```
-
-Test
---------------------
-```
-// Do prediction
-val predict = testDf
-  .select($"target", modelUdf($"features").as("predicted"))
-```
-
-Evaluation
---------------------
-```
-predict
-  .groupBy().agg(Map("target"->"avg", "predicted"->"avg"))
-  .show()
-```

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/f9f9799d/spark/spark-2.0/tutorials/binary_classification.md
----------------------------------------------------------------------
diff --git a/spark/spark-2.0/tutorials/binary_classification.md b/spark/spark-2.0/tutorials/binary_classification.md
deleted file mode 100644
index 22b1c29..0000000
--- a/spark/spark-2.0/tutorials/binary_classification.md
+++ /dev/null
@@ -1,71 +0,0 @@
-This tutorial uses [9a binary classification](https://github.com/myui/hivemall/wiki#a9a-binary-classification) as a reference.
-
-Data preparation
---------------------
-```
-// Fetch training data
-# wget http://www.csie.ntu.edu.tw/~cjlin/libsvmtools/datasets/binary/a9a
-# wget http://www.csie.ntu.edu.tw/~cjlin/libsvmtools/datasets/binary/a9a.t
-
-// Fetch a script to normalize the data
-# wget https://raw.githubusercontent.com/maropu/hivemall-spark/master/scripts/misc/conv.awk
-# awk -f conv.awk a9a | sed -e "s/+1/1/" | sed -e "s/-1/0/" > a9a.train
-# awk -f conv.awk a9a.t | sed -e "s/+1/1/" | sed -e "s/-1/0/" > a9a.test
-
-// Fetch an initialization script for hivemall-spark
-# wget https://raw.githubusercontent.com/maropu/hivemall-spark/master/scripts/ddl/define-dfs.sh
-
-// Invoke a spark-shell with hivemall-spark
-# bin/spark-shell --packages maropu:hivemall-spark:0.0.6 --master=local-cluster[2,1,1024] --conf spark.executor.memory=1024
-
-scala> :load define-dfs.sh
-
-scala> :paste
-
-// Load the training data as a RDD
-val trainRdd = sc.textFile("a9a.train")
-  .map(HmLabeledPoint.parse)
-
-// Create the DataFrame that has exactly 2 partitions and
-// amplify the data by 3 times.
-val trainDf = sqlContext.createDataFrame(trainRdd)
-  .coalesce(2).part_amplify(3)
-
-// Load the test data as a RDD
-val testRdd = sc.textFile("a9a.test")
-  .map(HmLabeledPoint.parse)
-
-// Transform into a DataFrame and transform features
-// into a Spark Vector type.
-val testDf = sqlContext.createDataFrame(testRdd)
-  .select($"label".as("target"), ft2vec($"features").as("features"))
-```
-
-Training (Logistic Regression)
---------------------
-```
-// Make a model from the training data
-val model = trainDf
-  .train_logregr(add_bias($"features"), $"label", "-total_steps 32561")
-  .groupby("feature").agg("weight" -> "avg")
-  .as("feature", "weight")
-
-val modelUdf = HivemallUtils
-  .funcModel(model)
-```
-
-Test
---------------------
-```
-// Do prediction
-val predict = testDf
-  .select($"target", sigmoid(modelUdf($"features")).as("prob"))
-  .select($"target", when($"prob" > 0.50, 1.0).otherwise(0.0).as("predict"), $"prob")
-  .cache
-```
-
-Evaluation
---------------------
-```
-(predict.where($"target" === $"predict").count + 0.0) / predict.count
-```

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/f9f9799d/spark/spark-2.0/tutorials/multiclass_classification.md
----------------------------------------------------------------------
diff --git a/spark/spark-2.0/tutorials/multiclass_classification.md b/spark/spark-2.0/tutorials/multiclass_classification.md
deleted file mode 100644
index 74cd3bc..0000000
--- a/spark/spark-2.0/tutorials/multiclass_classification.md
+++ /dev/null
@@ -1,77 +0,0 @@
-This tutorial uses [news20 multiclass classification](https://github.com/myui/hivemall/wiki#news20-multiclass-classification) as a reference.
-
-Data preparation
---------------------
-```
-// Fetch training and test data
-# wget http://www.csie.ntu.edu.tw/~cjlin/libsvmtools/datasets/multiclass/news20.scale.bz2
-# wget http://www.csie.ntu.edu.tw/~cjlin/libsvmtools/datasets/multiclass/news20.t.scale.bz2
-
-// Fetch a script to normalize the data
-# wget https://raw.githubusercontent.com/maropu/hivemall-spark/master/scripts/misc/conv.awk
-# bunzip2 -c news20.scale.bz2 | awk -f conv.awk > news20.train
-# bunzip2 -c news20.t.scale.bz2 | awk -f conv.awk > news20.test
-
-// Fetch an initialization script for hivemall-spark
-# wget https://raw.githubusercontent.com/maropu/hivemall-spark/master/scripts/ddl/define-dfs.sh
-
-// Invoke a spark-shell with hivemall-spark
-# bin/spark-shell --packages maropu:hivemall-spark:0.0.6 --master=local-cluster[2,1,1024] --conf spark.executor.memory=1024
-
-scala> :load define-dfs.sh
-
-scala> :paste
-
-// Load the training data as a RDD
-val trainRdd = sc.textFile("news20.train")
-  .map(HmLabeledPoint.parse)
-
-// Create the DataFrame that has exactly 2 partitions and
-// amplify the data by 3 times.
-val trainDf = sqlContext.createDataFrame(trainRdd)
-  .coalesce(2).part_amplify(3)
-
-// Load the test data as a RDD
-val testRdd = sc.textFile("news20.test")
-  .map(HmLabeledPoint.parse)
-
-// Transform into a DataFrame and transform features
-// into a Spark Vector type.
-val testDf = sqlContext.createDataFrame(testRdd)
-  .select(rowid(), $"label".cast(IntegerType).as("target"), $"features")
-  .cache
-
-val testDf_exploded = testDf.explode_array($"features")
-  .select($"rowid", $"target", extract_feature($"feature"), extract_weight($"feature"))
-```
-
-Training (CW)
---------------------
-```
-// Make a model from the training data
-val model = trainDf
-  .train_multiclass_cw(add_bias($"features"), $"label".cast(IntegerType))
-  .groupby("label", "feature").argmin_kld("weight", "conv")
-  .as("label", "feature", "weight")
-```
-
-Test
---------------------
-```
-// Do prediction
-val predict = testDf_exploded
-  .join(model, testDf_exploded("feature") === model("feature"), "LEFT_OUTER")
-  .select($"rowid", $"label".cast(StringType).as("label"), ($"weight" * $"value").as("value"))
-  .groupby("rowid", "label").sum("value")
-  .groupby("rowid").maxrow("SUM(value)", "label")
-  .as("rowid", "r")
-  .select($"rowid", $"r.col0".as("score"), $"r.col1".as("predict"))
-  .cache
-```
-
-Evaluation
---------------------
-```
-val joinPredicate = (testDf("rowid") === predict("rowid")).and(testDf("target") === predict("predict"))
-(testDf.join(predict, joinPredicate, "INNER").count + 0.0) / testDf.count
-```

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/f9f9799d/spark/spark-2.0/tutorials/randomforest.md
----------------------------------------------------------------------
diff --git a/spark/spark-2.0/tutorials/randomforest.md b/spark/spark-2.0/tutorials/randomforest.md
deleted file mode 100644
index 350e19a..0000000
--- a/spark/spark-2.0/tutorials/randomforest.md
+++ /dev/null
@@ -1,86 +0,0 @@
-This tutorial uses [Kaggle Titanic binary classification](https://github.com/myui/hivemall/wiki/Kaggle-Titanic-binary-classification-using-Random-Forest) as a reference.
-
-Data preparation
---------------------
-```
-// Fetch training and test data in Kaggle(https://www.kaggle.com/c/titanic/data), train.csv and test.csv
-
-// Fetch an initialization script for hivemall-spark
-# wget https://raw.githubusercontent.com/maropu/hivemall-spark/master/scripts/ddl/define-dfs.sh
-
-// Invoke a spark-shell with spark-csv and hivemall-spark
-# bin/spark-shell --packages com.databricks:spark-csv_2.10:1.4.0,maropu:hivemall-spark:0.0.6
-
-scala> :load define-dfs.sh
-
-scala> :paste
-
-// Load the training data as a DataFrame
-val trainCsvDf = sqlContext
-  .read
-  .format("com.databricks.spark.csv")
-  .option("header", "true")
-  .option("inferSchema", "true")
-  .load("train.csv")
-  .cache // Cached for second use
-
-val trainQuantifiedDf = trainCsvDf
-  .quantify(true.as("output") +: trainCsvDf.cols: _*)
-  // Rename output columns for readability
-  .as("passengerid", "survived", "pclass", "name", "sex", "age", "sibsp", "parch", "ticket", "fare", "cabin", "embarked")
-  .sort($"passengerid".asc)
-
-val trainDf = trainQuantifiedDf
-  .select(
-      $"passengerid",
-      array(trainQuantifiedDf.cols.drop(2): _*).as("features"),
-      $"survived"
-    )
-
-// Load the test data as a DataFrame
-val testCsvDf = sqlContext
-  .read
-  .format("com.databricks.spark.csv")
-  .option("header", "true")
-  .option("inferSchema", "true")
-  .load("test.csv")
-
-val testQuantifiedDf = testCsvDf
-  .select(Seq(1.as("train_first"), true.as("output"), $"PassengerId") ++ testCsvDf.cols.drop(1): _*)
-  .unionAll(
-      trainCsvDf.select(Seq(0.as("train_first"), false.as("output"), $"PassengerId") ++ trainCsvDf.cols.drop(2): _*)
-    )
-  .sort($"train_first".asc, $"PassengerId".asc)
-  .quantify($"output" +: testCsvDf.cols: _*)
-  // Rename output columns for readability
-  .as("passengerid", "pclass", "name", "sex", "age", "sibsp", "parch", "ticket", "fare", "cabin", "embarked")
-
-val testDf = testQuantifiedDf
-  .select($"passengerid", array(testQuantifiedDf.cols.drop(1): _*).as("features"))
-```
-
-Training
---------------------
-```
-// Make a model from the training data
-val model = trainDf
-  .coalesce(4)
-  .train_randomforest_classifier($"features", $"survived", "-trees 400")
-```
-
-Test
---------------------
-```
-// Do prediction
-model
-  .coalesce(4)
-  .join(testDf)
-  .select(
-      testDf("passengerid"),
-      tree_predict(model("model_id"), model("model_type"), model("pred_model"), testDf("features"), true).as("predicted")
-    )
-  .groupby($"passengerid").rf_ensemble("predicted")
-  .as("passengerid", "predicted")
-  .select($"passengerid", $"predicted.label")
-  .show
-```

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/f9f9799d/spark/spark-2.0/tutorials/regression.md
----------------------------------------------------------------------
diff --git a/spark/spark-2.0/tutorials/regression.md b/spark/spark-2.0/tutorials/regression.md
deleted file mode 100644
index 5856373..0000000
--- a/spark/spark-2.0/tutorials/regression.md
+++ /dev/null
@@ -1,71 +0,0 @@
-This tutorial uses [E2006 tfidf regression](https://github.com/myui/hivemall/wiki#e2006-tfidf-regression) as a reference.
-
-Data preparation
---------------------
-```
-// Fetch training and test data
-# wget http://www.csie.ntu.edu.tw/~cjlin/libsvmtools/datasets/regression/E2006.train.bz2
-# wget http://www.csie.ntu.edu.tw/~cjlin/libsvmtools/datasets/regression/E2006.test.bz2
-
-// Fetch a script to normalize the data
-# wget https://raw.githubusercontent.com/maropu/hivemall-spark/master/scripts/misc/conv.awk
-# bunzip2 -c E2006.train.bz2 | awk -f conv.awk > E2006.train.lp
-# bunzip2 -c E2006.test.bz2 | awk -f conv.awk > E2006.test.lp
-
-// Fetch an initialization script for hivemall-spark
-# wget https://raw.githubusercontent.com/maropu/hivemall-spark/master/scripts/ddl/define-dfs.sh
-
-// Invoke a spark-shell with hivemall-spark
-# bin/spark-shell --packages maropu:hivemall-spark:0.0.6 --master=local-cluster[2,1,1024] --conf spark.executor.memory=1024
-
-scala> :load define-dfs.sh
-
-scala> :paste
-
-// Load the training data as a RDD
-val trainRdd = sc.textFile("E2006.train.lp")
-  .map(HmLabeledPoint.parse)
-
-// Create the DataFrame that has exactly 2 partitions and
-// amplify the data by 3 times.
-val trainDf = sqlContext.createDataFrame(trainRdd)
-  .coalesce(2).part_amplify(3)
-
-// Load the test data as a RDD
-val testRdd = sc.textFile("E2006.test.lp")
-  .map(HmLabeledPoint.parse)
-
-// Transform into a DataFrame and transform features
-// into a Spark Vector type.
-val testDf = sqlContext.createDataFrame(testRdd)
-  .select($"label".as("target"), ft2vec($"features").as("features"))
-```
-
-Training (PA1)
---------------------
-```
-// Make a model from the training data
-val model = trainDf
-  .train_pa1_regr(add_bias($"features"), $"label")
-  .groupby("feature").agg("weight" -> "avg")
-  .as("feature", "weight")
-
-val modelUdf = HivemallUtils
-  .funcModel(model)
-```
-
-Test
---------------------
-```
-// Do prediction
-val predict = testDf
-  .select($"target", modelUdf($"features").as("predicted"))
-```
-
-Evaluation
---------------------
-```
-predict
-  .groupBy().agg(Map("target"->"avg", "predicted"->"avg"))
-  .show()
-```


[07/50] [abbrv] incubator-hivemall git commit: Re-enabled xgboost to avoid TravisCI error

Posted by my...@apache.org.
Re-enabled xgboost to avoid TravisCI error

Project: http://git-wip-us.apache.org/repos/asf/incubator-hivemall/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-hivemall/commit/34c2b0b3
Tree: http://git-wip-us.apache.org/repos/asf/incubator-hivemall/tree/34c2b0b3
Diff: http://git-wip-us.apache.org/repos/asf/incubator-hivemall/diff/34c2b0b3

Branch: refs/heads/master
Commit: 34c2b0b38b5cfd9044af60301b9875fd1601eeb9
Parents: eee7930
Author: myui <yu...@gmail.com>
Authored: Sun Oct 23 04:34:16 2016 +0900
Committer: myui <yu...@gmail.com>
Committed: Sun Oct 23 04:34:16 2016 +0900

----------------------------------------------------------------------
 pom.xml | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/34c2b0b3/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 265ed67..82496cb 100644
--- a/pom.xml
+++ b/pom.xml
@@ -206,7 +206,7 @@
 	<modules>
 		<module>core</module>
 		<module>nlp</module>
-		<!--<module>xgboost</module>-->
+		<module>xgboost</module>
 		<module>mixserv</module>
 	</modules>
 


[20/50] [abbrv] incubator-hivemall git commit: Updated license headers

Posted by my...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/ftvec/ExtractFeatureUDF.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/ftvec/ExtractFeatureUDF.java b/core/src/main/java/hivemall/ftvec/ExtractFeatureUDF.java
index 91d797f..5c9ff1e 100644
--- a/core/src/main/java/hivemall/ftvec/ExtractFeatureUDF.java
+++ b/core/src/main/java/hivemall/ftvec/ExtractFeatureUDF.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.ftvec;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/ftvec/ExtractWeightUDF.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/ftvec/ExtractWeightUDF.java b/core/src/main/java/hivemall/ftvec/ExtractWeightUDF.java
index ad0fcf5..01d4c01 100644
--- a/core/src/main/java/hivemall/ftvec/ExtractWeightUDF.java
+++ b/core/src/main/java/hivemall/ftvec/ExtractWeightUDF.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.ftvec;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/ftvec/FeatureIndexUDF.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/ftvec/FeatureIndexUDF.java b/core/src/main/java/hivemall/ftvec/FeatureIndexUDF.java
index 4126481..9ffe6c6 100644
--- a/core/src/main/java/hivemall/ftvec/FeatureIndexUDF.java
+++ b/core/src/main/java/hivemall/ftvec/FeatureIndexUDF.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.ftvec;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/ftvec/FeatureUDF.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/ftvec/FeatureUDF.java b/core/src/main/java/hivemall/ftvec/FeatureUDF.java
index b5bd036..b44459e 100644
--- a/core/src/main/java/hivemall/ftvec/FeatureUDF.java
+++ b/core/src/main/java/hivemall/ftvec/FeatureUDF.java
@@ -1,19 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.ftvec;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/ftvec/SortByFeatureUDF.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/ftvec/SortByFeatureUDF.java b/core/src/main/java/hivemall/ftvec/SortByFeatureUDF.java
index 0e4abfa..7a8f750 100644
--- a/core/src/main/java/hivemall/ftvec/SortByFeatureUDF.java
+++ b/core/src/main/java/hivemall/ftvec/SortByFeatureUDF.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.ftvec;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/ftvec/amplify/AmplifierUDTF.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/ftvec/amplify/AmplifierUDTF.java b/core/src/main/java/hivemall/ftvec/amplify/AmplifierUDTF.java
index 8e82133..46c23ab 100644
--- a/core/src/main/java/hivemall/ftvec/amplify/AmplifierUDTF.java
+++ b/core/src/main/java/hivemall/ftvec/amplify/AmplifierUDTF.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.ftvec.amplify;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/ftvec/amplify/RandomAmplifierUDTF.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/ftvec/amplify/RandomAmplifierUDTF.java b/core/src/main/java/hivemall/ftvec/amplify/RandomAmplifierUDTF.java
index 9f1ecb9..687d69a 100644
--- a/core/src/main/java/hivemall/ftvec/amplify/RandomAmplifierUDTF.java
+++ b/core/src/main/java/hivemall/ftvec/amplify/RandomAmplifierUDTF.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.ftvec.amplify;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/ftvec/binning/BuildBinsUDAF.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/ftvec/binning/BuildBinsUDAF.java b/core/src/main/java/hivemall/ftvec/binning/BuildBinsUDAF.java
index e2eada8..508d37c 100644
--- a/core/src/main/java/hivemall/ftvec/binning/BuildBinsUDAF.java
+++ b/core/src/main/java/hivemall/ftvec/binning/BuildBinsUDAF.java
@@ -1,19 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.ftvec.binning;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/ftvec/binning/FeatureBinningUDF.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/ftvec/binning/FeatureBinningUDF.java b/core/src/main/java/hivemall/ftvec/binning/FeatureBinningUDF.java
index 570a30f..3176490 100644
--- a/core/src/main/java/hivemall/ftvec/binning/FeatureBinningUDF.java
+++ b/core/src/main/java/hivemall/ftvec/binning/FeatureBinningUDF.java
@@ -1,19 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.ftvec.binning;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/ftvec/binning/NumericHistogram.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/ftvec/binning/NumericHistogram.java b/core/src/main/java/hivemall/ftvec/binning/NumericHistogram.java
index f529562..7e9a2c1 100644
--- a/core/src/main/java/hivemall/ftvec/binning/NumericHistogram.java
+++ b/core/src/main/java/hivemall/ftvec/binning/NumericHistogram.java
@@ -1,19 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.ftvec.binning;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/ftvec/conv/ConvertToDenseModelUDAF.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/ftvec/conv/ConvertToDenseModelUDAF.java b/core/src/main/java/hivemall/ftvec/conv/ConvertToDenseModelUDAF.java
index a4a7fc4..008dd3a 100644
--- a/core/src/main/java/hivemall/ftvec/conv/ConvertToDenseModelUDAF.java
+++ b/core/src/main/java/hivemall/ftvec/conv/ConvertToDenseModelUDAF.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.ftvec.conv;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/ftvec/conv/QuantifyColumnsUDTF.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/ftvec/conv/QuantifyColumnsUDTF.java b/core/src/main/java/hivemall/ftvec/conv/QuantifyColumnsUDTF.java
index 94b59ee..e8214a1 100644
--- a/core/src/main/java/hivemall/ftvec/conv/QuantifyColumnsUDTF.java
+++ b/core/src/main/java/hivemall/ftvec/conv/QuantifyColumnsUDTF.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.ftvec.conv;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/ftvec/conv/ToDenseFeaturesUDF.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/ftvec/conv/ToDenseFeaturesUDF.java b/core/src/main/java/hivemall/ftvec/conv/ToDenseFeaturesUDF.java
index 70b57ba..7670ce1 100644
--- a/core/src/main/java/hivemall/ftvec/conv/ToDenseFeaturesUDF.java
+++ b/core/src/main/java/hivemall/ftvec/conv/ToDenseFeaturesUDF.java
@@ -1,19 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.ftvec.conv;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/ftvec/conv/ToSparseFeaturesUDF.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/ftvec/conv/ToSparseFeaturesUDF.java b/core/src/main/java/hivemall/ftvec/conv/ToSparseFeaturesUDF.java
index 0a502b4..9bf1e69 100644
--- a/core/src/main/java/hivemall/ftvec/conv/ToSparseFeaturesUDF.java
+++ b/core/src/main/java/hivemall/ftvec/conv/ToSparseFeaturesUDF.java
@@ -1,19 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.ftvec.conv;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/ftvec/hashing/ArrayHashValuesUDF.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/ftvec/hashing/ArrayHashValuesUDF.java b/core/src/main/java/hivemall/ftvec/hashing/ArrayHashValuesUDF.java
index 498359d..b8c0c13 100644
--- a/core/src/main/java/hivemall/ftvec/hashing/ArrayHashValuesUDF.java
+++ b/core/src/main/java/hivemall/ftvec/hashing/ArrayHashValuesUDF.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.ftvec.hashing;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/ftvec/hashing/ArrayPrefixedHashValuesUDF.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/ftvec/hashing/ArrayPrefixedHashValuesUDF.java b/core/src/main/java/hivemall/ftvec/hashing/ArrayPrefixedHashValuesUDF.java
index 4499af6..033df8d 100644
--- a/core/src/main/java/hivemall/ftvec/hashing/ArrayPrefixedHashValuesUDF.java
+++ b/core/src/main/java/hivemall/ftvec/hashing/ArrayPrefixedHashValuesUDF.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.ftvec.hashing;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/ftvec/hashing/FeatureHashingUDF.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/ftvec/hashing/FeatureHashingUDF.java b/core/src/main/java/hivemall/ftvec/hashing/FeatureHashingUDF.java
index c5f9ce0..ce6565c 100644
--- a/core/src/main/java/hivemall/ftvec/hashing/FeatureHashingUDF.java
+++ b/core/src/main/java/hivemall/ftvec/hashing/FeatureHashingUDF.java
@@ -1,19 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.ftvec.hashing;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/ftvec/hashing/MurmurHash3UDF.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/ftvec/hashing/MurmurHash3UDF.java b/core/src/main/java/hivemall/ftvec/hashing/MurmurHash3UDF.java
index 7006186..086ca34 100644
--- a/core/src/main/java/hivemall/ftvec/hashing/MurmurHash3UDF.java
+++ b/core/src/main/java/hivemall/ftvec/hashing/MurmurHash3UDF.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.ftvec.hashing;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/ftvec/hashing/Sha1UDF.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/ftvec/hashing/Sha1UDF.java b/core/src/main/java/hivemall/ftvec/hashing/Sha1UDF.java
index 43907bb..021f462 100644
--- a/core/src/main/java/hivemall/ftvec/hashing/Sha1UDF.java
+++ b/core/src/main/java/hivemall/ftvec/hashing/Sha1UDF.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.ftvec.hashing;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/ftvec/pairing/PolynomialFeaturesUDF.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/ftvec/pairing/PolynomialFeaturesUDF.java b/core/src/main/java/hivemall/ftvec/pairing/PolynomialFeaturesUDF.java
index ab5b1ef..742e76d 100644
--- a/core/src/main/java/hivemall/ftvec/pairing/PolynomialFeaturesUDF.java
+++ b/core/src/main/java/hivemall/ftvec/pairing/PolynomialFeaturesUDF.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.ftvec.pairing;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/ftvec/pairing/PoweredFeaturesUDF.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/ftvec/pairing/PoweredFeaturesUDF.java b/core/src/main/java/hivemall/ftvec/pairing/PoweredFeaturesUDF.java
index 6e0cea5..a40693a 100644
--- a/core/src/main/java/hivemall/ftvec/pairing/PoweredFeaturesUDF.java
+++ b/core/src/main/java/hivemall/ftvec/pairing/PoweredFeaturesUDF.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.ftvec.pairing;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/ftvec/ranking/BprSamplingUDTF.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/ftvec/ranking/BprSamplingUDTF.java b/core/src/main/java/hivemall/ftvec/ranking/BprSamplingUDTF.java
index 7d2db3f..8e84bd8 100644
--- a/core/src/main/java/hivemall/ftvec/ranking/BprSamplingUDTF.java
+++ b/core/src/main/java/hivemall/ftvec/ranking/BprSamplingUDTF.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.ftvec.ranking;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/ftvec/ranking/ItemPairsSamplingUDTF.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/ftvec/ranking/ItemPairsSamplingUDTF.java b/core/src/main/java/hivemall/ftvec/ranking/ItemPairsSamplingUDTF.java
index 59699ae..64ef4ba 100644
--- a/core/src/main/java/hivemall/ftvec/ranking/ItemPairsSamplingUDTF.java
+++ b/core/src/main/java/hivemall/ftvec/ranking/ItemPairsSamplingUDTF.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.ftvec.ranking;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/ftvec/ranking/PerEventPositiveOnlyFeedback.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/ftvec/ranking/PerEventPositiveOnlyFeedback.java b/core/src/main/java/hivemall/ftvec/ranking/PerEventPositiveOnlyFeedback.java
index 447ab19..b5afb99 100644
--- a/core/src/main/java/hivemall/ftvec/ranking/PerEventPositiveOnlyFeedback.java
+++ b/core/src/main/java/hivemall/ftvec/ranking/PerEventPositiveOnlyFeedback.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.ftvec.ranking;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/ftvec/ranking/PopulateNotInUDTF.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/ftvec/ranking/PopulateNotInUDTF.java b/core/src/main/java/hivemall/ftvec/ranking/PopulateNotInUDTF.java
index efbe124..29529e1 100644
--- a/core/src/main/java/hivemall/ftvec/ranking/PopulateNotInUDTF.java
+++ b/core/src/main/java/hivemall/ftvec/ranking/PopulateNotInUDTF.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.ftvec.ranking;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/ftvec/ranking/PositiveOnlyFeedback.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/ftvec/ranking/PositiveOnlyFeedback.java b/core/src/main/java/hivemall/ftvec/ranking/PositiveOnlyFeedback.java
index 9b96af5..908a0b7 100644
--- a/core/src/main/java/hivemall/ftvec/ranking/PositiveOnlyFeedback.java
+++ b/core/src/main/java/hivemall/ftvec/ranking/PositiveOnlyFeedback.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.ftvec.ranking;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/ftvec/scaling/L2NormalizationUDF.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/ftvec/scaling/L2NormalizationUDF.java b/core/src/main/java/hivemall/ftvec/scaling/L2NormalizationUDF.java
index ffb8bd6..fe4acf7 100644
--- a/core/src/main/java/hivemall/ftvec/scaling/L2NormalizationUDF.java
+++ b/core/src/main/java/hivemall/ftvec/scaling/L2NormalizationUDF.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.ftvec.scaling;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/ftvec/scaling/RescaleUDF.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/ftvec/scaling/RescaleUDF.java b/core/src/main/java/hivemall/ftvec/scaling/RescaleUDF.java
index a21b8a2..21a30d5 100644
--- a/core/src/main/java/hivemall/ftvec/scaling/RescaleUDF.java
+++ b/core/src/main/java/hivemall/ftvec/scaling/RescaleUDF.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.ftvec.scaling;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/ftvec/scaling/ZScoreUDF.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/ftvec/scaling/ZScoreUDF.java b/core/src/main/java/hivemall/ftvec/scaling/ZScoreUDF.java
index 599af80..93ab7ed 100644
--- a/core/src/main/java/hivemall/ftvec/scaling/ZScoreUDF.java
+++ b/core/src/main/java/hivemall/ftvec/scaling/ZScoreUDF.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.ftvec.scaling;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/ftvec/text/TermFrequencyUDAF.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/ftvec/text/TermFrequencyUDAF.java b/core/src/main/java/hivemall/ftvec/text/TermFrequencyUDAF.java
index 0477a23..95a881b 100644
--- a/core/src/main/java/hivemall/ftvec/text/TermFrequencyUDAF.java
+++ b/core/src/main/java/hivemall/ftvec/text/TermFrequencyUDAF.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.ftvec.text;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/ftvec/trans/BinarizeLabelUDTF.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/ftvec/trans/BinarizeLabelUDTF.java b/core/src/main/java/hivemall/ftvec/trans/BinarizeLabelUDTF.java
index f3f7a82..6f8fc0a 100644
--- a/core/src/main/java/hivemall/ftvec/trans/BinarizeLabelUDTF.java
+++ b/core/src/main/java/hivemall/ftvec/trans/BinarizeLabelUDTF.java
@@ -1,19 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.ftvec.trans;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/ftvec/trans/CategoricalFeaturesUDF.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/ftvec/trans/CategoricalFeaturesUDF.java b/core/src/main/java/hivemall/ftvec/trans/CategoricalFeaturesUDF.java
index 1838d04..98617bd 100644
--- a/core/src/main/java/hivemall/ftvec/trans/CategoricalFeaturesUDF.java
+++ b/core/src/main/java/hivemall/ftvec/trans/CategoricalFeaturesUDF.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.ftvec.trans;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/ftvec/trans/FFMFeaturesUDF.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/ftvec/trans/FFMFeaturesUDF.java b/core/src/main/java/hivemall/ftvec/trans/FFMFeaturesUDF.java
index 1a9cfae..c98ffda 100644
--- a/core/src/main/java/hivemall/ftvec/trans/FFMFeaturesUDF.java
+++ b/core/src/main/java/hivemall/ftvec/trans/FFMFeaturesUDF.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.ftvec.trans;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/ftvec/trans/IndexedFeatures.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/ftvec/trans/IndexedFeatures.java b/core/src/main/java/hivemall/ftvec/trans/IndexedFeatures.java
index 51c92b6..eb61006 100644
--- a/core/src/main/java/hivemall/ftvec/trans/IndexedFeatures.java
+++ b/core/src/main/java/hivemall/ftvec/trans/IndexedFeatures.java
@@ -1,19 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.ftvec.trans;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/ftvec/trans/OnehotEncodingUDAF.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/ftvec/trans/OnehotEncodingUDAF.java b/core/src/main/java/hivemall/ftvec/trans/OnehotEncodingUDAF.java
index 1d01130..704a4c7 100644
--- a/core/src/main/java/hivemall/ftvec/trans/OnehotEncodingUDAF.java
+++ b/core/src/main/java/hivemall/ftvec/trans/OnehotEncodingUDAF.java
@@ -1,19 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2016 Makoto YUI
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.ftvec.trans;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/ftvec/trans/QuantifiedFeaturesUDTF.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/ftvec/trans/QuantifiedFeaturesUDTF.java b/core/src/main/java/hivemall/ftvec/trans/QuantifiedFeaturesUDTF.java
index 2256697..2886996 100644
--- a/core/src/main/java/hivemall/ftvec/trans/QuantifiedFeaturesUDTF.java
+++ b/core/src/main/java/hivemall/ftvec/trans/QuantifiedFeaturesUDTF.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.ftvec.trans;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/ftvec/trans/QuantitativeFeaturesUDF.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/ftvec/trans/QuantitativeFeaturesUDF.java b/core/src/main/java/hivemall/ftvec/trans/QuantitativeFeaturesUDF.java
index 14ea83f..43f837f 100644
--- a/core/src/main/java/hivemall/ftvec/trans/QuantitativeFeaturesUDF.java
+++ b/core/src/main/java/hivemall/ftvec/trans/QuantitativeFeaturesUDF.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.ftvec.trans;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/ftvec/trans/VectorizeFeaturesUDF.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/ftvec/trans/VectorizeFeaturesUDF.java b/core/src/main/java/hivemall/ftvec/trans/VectorizeFeaturesUDF.java
index 210b2af..48bf126 100644
--- a/core/src/main/java/hivemall/ftvec/trans/VectorizeFeaturesUDF.java
+++ b/core/src/main/java/hivemall/ftvec/trans/VectorizeFeaturesUDF.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.ftvec.trans;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/knn/distance/AngularDistanceUDF.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/knn/distance/AngularDistanceUDF.java b/core/src/main/java/hivemall/knn/distance/AngularDistanceUDF.java
index 1dc6184..c2f864f 100644
--- a/core/src/main/java/hivemall/knn/distance/AngularDistanceUDF.java
+++ b/core/src/main/java/hivemall/knn/distance/AngularDistanceUDF.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.knn.distance;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/knn/distance/CosineDistanceUDF.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/knn/distance/CosineDistanceUDF.java b/core/src/main/java/hivemall/knn/distance/CosineDistanceUDF.java
index 7305d05..8c36660 100644
--- a/core/src/main/java/hivemall/knn/distance/CosineDistanceUDF.java
+++ b/core/src/main/java/hivemall/knn/distance/CosineDistanceUDF.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.knn.distance;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/knn/distance/EuclidDistanceUDF.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/knn/distance/EuclidDistanceUDF.java b/core/src/main/java/hivemall/knn/distance/EuclidDistanceUDF.java
index fd95541..26401f6 100644
--- a/core/src/main/java/hivemall/knn/distance/EuclidDistanceUDF.java
+++ b/core/src/main/java/hivemall/knn/distance/EuclidDistanceUDF.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.knn.distance;
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/d806bf45/core/src/main/java/hivemall/knn/distance/HammingDistanceUDF.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/knn/distance/HammingDistanceUDF.java b/core/src/main/java/hivemall/knn/distance/HammingDistanceUDF.java
index 1cdedc9..c420d8b 100644
--- a/core/src/main/java/hivemall/knn/distance/HammingDistanceUDF.java
+++ b/core/src/main/java/hivemall/knn/distance/HammingDistanceUDF.java
@@ -1,20 +1,20 @@
 /*
- * Hivemall: Hive scalable Machine Learning Library
+ * 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
  *
- * Copyright (C) 2015 Makoto YUI
- * Copyright (C) 2013-2015 National Institute of Advanced Industrial Science and Technology (AIST)
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * Licensed 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.
+ * 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.knn.distance;
 



[26/50] [abbrv] incubator-hivemall git commit: Updated license headers of *.md

Posted by my...@apache.org.
Updated license headers of *.md

Project: http://git-wip-us.apache.org/repos/asf/incubator-hivemall/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-hivemall/commit/fc97a52e
Tree: http://git-wip-us.apache.org/repos/asf/incubator-hivemall/tree/fc97a52e
Diff: http://git-wip-us.apache.org/repos/asf/incubator-hivemall/diff/fc97a52e

Branch: refs/heads/master
Commit: fc97a52eadd512edfb368c6a6b81455a99a534da
Parents: 24f1779
Author: myui <yu...@gmail.com>
Authored: Mon Nov 7 14:49:10 2016 +0900
Committer: myui <yu...@gmail.com>
Committed: Mon Nov 7 14:49:10 2016 +0900

----------------------------------------------------------------------
 README.md                                       | 23 ++++++++++++++++++--
 docs/gitbook/README.md                          | 19 ++++++++++++++++
 docs/gitbook/SUMMARY.md                         | 19 ++++++++++++++++
 docs/gitbook/anomaly/lof.md                     | 19 ++++++++++++++++
 docs/gitbook/binaryclass/a9a.md                 | 19 ++++++++++++++++
 docs/gitbook/binaryclass/a9a_dataset.md         | 19 ++++++++++++++++
 docs/gitbook/binaryclass/a9a_lr.md              | 19 ++++++++++++++++
 docs/gitbook/binaryclass/a9a_minibatch.md       | 19 ++++++++++++++++
 docs/gitbook/binaryclass/kdd2010a.md            | 19 ++++++++++++++++
 docs/gitbook/binaryclass/kdd2010a_dataset.md    | 19 ++++++++++++++++
 docs/gitbook/binaryclass/kdd2010a_scw.md        | 19 ++++++++++++++++
 docs/gitbook/binaryclass/kdd2010b.md            | 19 ++++++++++++++++
 docs/gitbook/binaryclass/kdd2010b_arow.md       | 19 ++++++++++++++++
 docs/gitbook/binaryclass/kdd2010b_dataset.md    | 19 ++++++++++++++++
 docs/gitbook/binaryclass/news20.md              | 19 ++++++++++++++++
 docs/gitbook/binaryclass/news20_adagrad.md      | 19 ++++++++++++++++
 docs/gitbook/binaryclass/news20_dataset.md      | 19 ++++++++++++++++
 docs/gitbook/binaryclass/news20_pa.md           | 19 ++++++++++++++++
 docs/gitbook/binaryclass/news20_scw.md          | 19 ++++++++++++++++
 docs/gitbook/binaryclass/webspam.md             | 19 ++++++++++++++++
 docs/gitbook/binaryclass/webspam_dataset.md     | 19 ++++++++++++++++
 docs/gitbook/binaryclass/webspam_scw.md         | 19 ++++++++++++++++
 docs/gitbook/eval/datagen.md                    | 19 ++++++++++++++++
 docs/gitbook/eval/lr_datagen.md                 | 19 ++++++++++++++++
 docs/gitbook/eval/stat_eval.md                  | 19 ++++++++++++++++
 docs/gitbook/ft_engineering/ft_trans.md         | 19 ++++++++++++++++
 docs/gitbook/ft_engineering/hashing.md          | 19 ++++++++++++++++
 docs/gitbook/ft_engineering/quantify.md         | 19 ++++++++++++++++
 docs/gitbook/ft_engineering/scaling.md          | 19 ++++++++++++++++
 docs/gitbook/ft_engineering/tfidf.md            | 19 ++++++++++++++++
 docs/gitbook/ft_engineering/vectorizer.md       | 19 ++++++++++++++++
 docs/gitbook/getting_started/README.md          | 19 ++++++++++++++++
 docs/gitbook/getting_started/input-format.md    | 19 ++++++++++++++++
 docs/gitbook/getting_started/installation.md    | 19 ++++++++++++++++
 .../getting_started/permanent-functions.md      | 19 ++++++++++++++++
 docs/gitbook/misc/generic_funcs.md              | 19 ++++++++++++++++
 docs/gitbook/misc/tokenizer.md                  | 19 ++++++++++++++++
 docs/gitbook/misc/topk.md                       | 19 ++++++++++++++++
 docs/gitbook/multiclass/iris.md                 | 19 ++++++++++++++++
 docs/gitbook/multiclass/iris_dataset.md         | 19 ++++++++++++++++
 docs/gitbook/multiclass/iris_randomforest.md    | 19 ++++++++++++++++
 docs/gitbook/multiclass/iris_scw.md             | 19 ++++++++++++++++
 docs/gitbook/multiclass/news20.md               | 19 ++++++++++++++++
 docs/gitbook/multiclass/news20_dataset.md       | 19 ++++++++++++++++
 docs/gitbook/multiclass/news20_ensemble.md      | 19 ++++++++++++++++
 .../multiclass/news20_one-vs-the-rest.md        | 19 ++++++++++++++++
 .../news20_one-vs-the-rest_dataset.md           | 19 ++++++++++++++++
 docs/gitbook/multiclass/news20_pa.md            | 19 ++++++++++++++++
 docs/gitbook/multiclass/news20_scw.md           | 19 ++++++++++++++++
 docs/gitbook/recommend/cf.md                    | 19 ++++++++++++++++
 docs/gitbook/recommend/item_based_cf.md         | 19 ++++++++++++++++
 docs/gitbook/recommend/movielens.md             | 19 ++++++++++++++++
 docs/gitbook/recommend/movielens_cv.md          | 19 ++++++++++++++++
 docs/gitbook/recommend/movielens_dataset.md     | 19 ++++++++++++++++
 docs/gitbook/recommend/movielens_fm.md          | 19 ++++++++++++++++
 docs/gitbook/recommend/movielens_mf.md          | 19 ++++++++++++++++
 docs/gitbook/recommend/news20.md                | 19 ++++++++++++++++
 docs/gitbook/recommend/news20_bbit_minhash.md   | 19 ++++++++++++++++
 docs/gitbook/recommend/news20_jaccard.md        | 19 ++++++++++++++++
 docs/gitbook/recommend/news20_knn.md            | 19 ++++++++++++++++
 docs/gitbook/regression/e2006.md                | 19 ++++++++++++++++
 docs/gitbook/regression/e2006_arow.md           | 19 ++++++++++++++++
 docs/gitbook/regression/e2006_dataset.md        | 19 ++++++++++++++++
 docs/gitbook/regression/kddcup12tr2.md          | 19 ++++++++++++++++
 docs/gitbook/regression/kddcup12tr2_adagrad.md  | 19 ++++++++++++++++
 docs/gitbook/regression/kddcup12tr2_dataset.md  | 19 ++++++++++++++++
 docs/gitbook/regression/kddcup12tr2_lr.md       | 19 ++++++++++++++++
 .../regression/kddcup12tr2_lr_amplify.md        | 19 ++++++++++++++++
 docs/gitbook/tips/README.md                     | 19 ++++++++++++++++
 docs/gitbook/tips/addbias.md                    | 19 ++++++++++++++++
 docs/gitbook/tips/emr.md                        | 19 ++++++++++++++++
 docs/gitbook/tips/ensemble_learning.md          | 19 ++++++++++++++++
 docs/gitbook/tips/general_tips.md               | 19 ++++++++++++++++
 docs/gitbook/tips/hadoop_tuning.md              | 19 ++++++++++++++++
 docs/gitbook/tips/mixserver.md                  | 19 ++++++++++++++++
 docs/gitbook/tips/rand_amplify.md               | 19 ++++++++++++++++
 docs/gitbook/tips/rowid.md                      | 19 ++++++++++++++++
 docs/gitbook/tips/rt_prediction.md              | 19 ++++++++++++++++
 docs/gitbook/troubleshooting/README.md          | 19 ++++++++++++++++
 docs/gitbook/troubleshooting/asterisk.md        | 19 ++++++++++++++++
 .../troubleshooting/mapjoin_classcastex.md      | 19 ++++++++++++++++
 .../troubleshooting/mapjoin_task_error.md       | 19 ++++++++++++++++
 docs/gitbook/troubleshooting/num_mappers.md     | 19 ++++++++++++++++
 docs/gitbook/troubleshooting/oom.md             | 19 ++++++++++++++++
 pom.xml                                         |  5 +++++
 resources/header-definition.xml                 | 12 ++++++++++
 86 files changed, 1615 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/fc97a52e/README.md
----------------------------------------------------------------------
diff --git a/README.md b/README.md
index 75ad6cd..79333aa 100644
--- a/README.md
+++ b/README.md
@@ -1,5 +1,24 @@
-Hivemall: Hive scalable machine learning library
-=================================================
+<!--
+  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.
+-->
+
+Apache Hivemall: Hive scalable machine learning library
+=======================================================
 [![Build Status](https://travis-ci.org/myui/hivemall.svg?branch=master)](https://travis-ci.org/myui/hivemall)
 [![Coversity](https://scan.coverity.com/projects/4549/badge.svg)](https://scan.coverity.com/projects/4549)
 [![Documentation Status](https://readthedocs.org/projects/hivemall-docs/badge/?version=latest)](https://readthedocs.org/projects/hivemall-docs/?badge=latest)

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/fc97a52e/docs/gitbook/README.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/README.md b/docs/gitbook/README.md
index 82602f8..7b61570 100644
--- a/docs/gitbook/README.md
+++ b/docs/gitbook/README.md
@@ -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.
+-->
+        
 # Introduction
 
 <div class="alert alert-info">

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/fc97a52e/docs/gitbook/SUMMARY.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/SUMMARY.md b/docs/gitbook/SUMMARY.md
index d85f952..7ef1b9b 100644
--- a/docs/gitbook/SUMMARY.md
+++ b/docs/gitbook/SUMMARY.md
@@ -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.
+-->
+        
 # Summary
 
 ## TABLE OF CONTENTS

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/fc97a52e/docs/gitbook/anomaly/lof.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/anomaly/lof.md b/docs/gitbook/anomaly/lof.md
index f8f0b61..48990f8 100644
--- a/docs/gitbook/anomaly/lof.md
+++ b/docs/gitbook/anomaly/lof.md
@@ -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.
+-->
+        
 This article introduce how to find outliers using [Local Outlier Detection (LOF)](http://en.wikipedia.org/wiki/Local_outlier_factor) on Hivemall.
 
 # Data Preparation

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/fc97a52e/docs/gitbook/binaryclass/a9a.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/binaryclass/a9a.md b/docs/gitbook/binaryclass/a9a.md
index e69de29..2959148 100644
--- a/docs/gitbook/binaryclass/a9a.md
+++ b/docs/gitbook/binaryclass/a9a.md
@@ -0,0 +1,19 @@
+<!--
+  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.
+-->
+        

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/fc97a52e/docs/gitbook/binaryclass/a9a_dataset.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/binaryclass/a9a_dataset.md b/docs/gitbook/binaryclass/a9a_dataset.md
index 28bcd57..76ccb0d 100644
--- a/docs/gitbook/binaryclass/a9a_dataset.md
+++ b/docs/gitbook/binaryclass/a9a_dataset.md
@@ -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.
+-->
+        
 a9a
 ===
 http://www.csie.ntu.edu.tw/~cjlin/libsvmtools/datasets/binary.html#a9a

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/fc97a52e/docs/gitbook/binaryclass/a9a_lr.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/binaryclass/a9a_lr.md b/docs/gitbook/binaryclass/a9a_lr.md
index 5029c49..17d91c0 100644
--- a/docs/gitbook/binaryclass/a9a_lr.md
+++ b/docs/gitbook/binaryclass/a9a_lr.md
@@ -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.
+-->
+        
 a9a
 ===
 http://www.csie.ntu.edu.tw/~cjlin/libsvmtools/datasets/binary.html#a9a

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/fc97a52e/docs/gitbook/binaryclass/a9a_minibatch.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/binaryclass/a9a_minibatch.md b/docs/gitbook/binaryclass/a9a_minibatch.md
index 714db6a..eaa7a06 100644
--- a/docs/gitbook/binaryclass/a9a_minibatch.md
+++ b/docs/gitbook/binaryclass/a9a_minibatch.md
@@ -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.
+-->
+        
 This page explains how to apply [Mini-Batch Gradient Descent](https://class.coursera.org/ml-003/lecture/106) for the training of logistic regression explained in [this example](https://github.com/myui/hivemall/wiki/a9a-binary-classification-(logistic-regression)). 
 
 See [this page](https://github.com/myui/hivemall/wiki/a9a-binary-classification-(logistic-regression)) first. This content depends on it.

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/fc97a52e/docs/gitbook/binaryclass/kdd2010a.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/binaryclass/kdd2010a.md b/docs/gitbook/binaryclass/kdd2010a.md
index e69de29..2959148 100644
--- a/docs/gitbook/binaryclass/kdd2010a.md
+++ b/docs/gitbook/binaryclass/kdd2010a.md
@@ -0,0 +1,19 @@
+<!--
+  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.
+-->
+        

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/fc97a52e/docs/gitbook/binaryclass/kdd2010a_dataset.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/binaryclass/kdd2010a_dataset.md b/docs/gitbook/binaryclass/kdd2010a_dataset.md
index 731d68b..ca221c3 100644
--- a/docs/gitbook/binaryclass/kdd2010a_dataset.md
+++ b/docs/gitbook/binaryclass/kdd2010a_dataset.md
@@ -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.
+-->
+        
 [http://www.csie.ntu.edu.tw/~cjlin/libsvmtools/datasets/binary.html#kdd2010 (algebra)](http://www.csie.ntu.edu.tw/~cjlin/libsvmtools/datasets/binary.html#kdd2010 (algebra))
 
 * # of classes: 2

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/fc97a52e/docs/gitbook/binaryclass/kdd2010a_scw.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/binaryclass/kdd2010a_scw.md b/docs/gitbook/binaryclass/kdd2010a_scw.md
index ee8fbba..5cb19fc 100644
--- a/docs/gitbook/binaryclass/kdd2010a_scw.md
+++ b/docs/gitbook/binaryclass/kdd2010a_scw.md
@@ -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.
+-->
+        
 # PA1
 ## Train
 ```sql

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/fc97a52e/docs/gitbook/binaryclass/kdd2010b.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/binaryclass/kdd2010b.md b/docs/gitbook/binaryclass/kdd2010b.md
index e69de29..2959148 100644
--- a/docs/gitbook/binaryclass/kdd2010b.md
+++ b/docs/gitbook/binaryclass/kdd2010b.md
@@ -0,0 +1,19 @@
+<!--
+  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.
+-->
+        

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/fc97a52e/docs/gitbook/binaryclass/kdd2010b_arow.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/binaryclass/kdd2010b_arow.md b/docs/gitbook/binaryclass/kdd2010b_arow.md
index 7ac845a..2ca0d90 100644
--- a/docs/gitbook/binaryclass/kdd2010b_arow.md
+++ b/docs/gitbook/binaryclass/kdd2010b_arow.md
@@ -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.
+-->
+        
 ## training
 ```sql
 -- SET mapred.reduce.tasks=32;

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/fc97a52e/docs/gitbook/binaryclass/kdd2010b_dataset.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/binaryclass/kdd2010b_dataset.md b/docs/gitbook/binaryclass/kdd2010b_dataset.md
index 5e26dba..41f0513 100644
--- a/docs/gitbook/binaryclass/kdd2010b_dataset.md
+++ b/docs/gitbook/binaryclass/kdd2010b_dataset.md
@@ -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.
+-->
+        
 [http://www.csie.ntu.edu.tw/~cjlin/libsvmtools/datasets/binary.html#kdd2010 (bridge to algebra)](http://www.csie.ntu.edu.tw/~cjlin/libsvmtools/datasets/binary.html#kdd2010 (bridge to algebra))
 
 * # of classes: 2

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/fc97a52e/docs/gitbook/binaryclass/news20.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/binaryclass/news20.md b/docs/gitbook/binaryclass/news20.md
index e69de29..2959148 100644
--- a/docs/gitbook/binaryclass/news20.md
+++ b/docs/gitbook/binaryclass/news20.md
@@ -0,0 +1,19 @@
+<!--
+  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.
+-->
+        

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/fc97a52e/docs/gitbook/binaryclass/news20_adagrad.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/binaryclass/news20_adagrad.md b/docs/gitbook/binaryclass/news20_adagrad.md
index 08e39df..cbcc0f2 100644
--- a/docs/gitbook/binaryclass/news20_adagrad.md
+++ b/docs/gitbook/binaryclass/news20_adagrad.md
@@ -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.
+-->
+        
 _Note that this feature is supported since Hivemall v0.3-beta2 or later._
 
 ## UDF preparation

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/fc97a52e/docs/gitbook/binaryclass/news20_dataset.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/binaryclass/news20_dataset.md b/docs/gitbook/binaryclass/news20_dataset.md
index 5ff80cd..87208cf 100644
--- a/docs/gitbook/binaryclass/news20_dataset.md
+++ b/docs/gitbook/binaryclass/news20_dataset.md
@@ -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.
+-->
+        
 Get the news20b dataset.
 http://www.csie.ntu.edu.tw/~cjlin/libsvmtools/datasets/binary.html#news20.binary
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/fc97a52e/docs/gitbook/binaryclass/news20_pa.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/binaryclass/news20_pa.md b/docs/gitbook/binaryclass/news20_pa.md
index 7763a15..df082b9 100644
--- a/docs/gitbook/binaryclass/news20_pa.md
+++ b/docs/gitbook/binaryclass/news20_pa.md
@@ -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.
+-->
+        
 ## UDF preparation
 ```
 delete jar /home/myui/tmp/hivemall.jar;

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/fc97a52e/docs/gitbook/binaryclass/news20_scw.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/binaryclass/news20_scw.md b/docs/gitbook/binaryclass/news20_scw.md
index 0b27dab..fa1da7f 100644
--- a/docs/gitbook/binaryclass/news20_scw.md
+++ b/docs/gitbook/binaryclass/news20_scw.md
@@ -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.
+-->
+        
 ## UDF preparation
 ```
 use news20;

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/fc97a52e/docs/gitbook/binaryclass/webspam.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/binaryclass/webspam.md b/docs/gitbook/binaryclass/webspam.md
index e69de29..2959148 100644
--- a/docs/gitbook/binaryclass/webspam.md
+++ b/docs/gitbook/binaryclass/webspam.md
@@ -0,0 +1,19 @@
+<!--
+  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.
+-->
+        

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/fc97a52e/docs/gitbook/binaryclass/webspam_dataset.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/binaryclass/webspam_dataset.md b/docs/gitbook/binaryclass/webspam_dataset.md
index 4686865..40cac07 100644
--- a/docs/gitbook/binaryclass/webspam_dataset.md
+++ b/docs/gitbook/binaryclass/webspam_dataset.md
@@ -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.
+-->
+        
 Get the dataset from 
 http://www.csie.ntu.edu.tw/~cjlin/libsvmtools/datasets/binary.html#webspam
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/fc97a52e/docs/gitbook/binaryclass/webspam_scw.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/binaryclass/webspam_scw.md b/docs/gitbook/binaryclass/webspam_scw.md
index 635b32d..cadd0ab 100644
--- a/docs/gitbook/binaryclass/webspam_scw.md
+++ b/docs/gitbook/binaryclass/webspam_scw.md
@@ -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.
+-->
+        
 # Preparation
 
 ```

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/fc97a52e/docs/gitbook/eval/datagen.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/eval/datagen.md b/docs/gitbook/eval/datagen.md
index e69de29..2959148 100644
--- a/docs/gitbook/eval/datagen.md
+++ b/docs/gitbook/eval/datagen.md
@@ -0,0 +1,19 @@
+<!--
+  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.
+-->
+        

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/fc97a52e/docs/gitbook/eval/lr_datagen.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/eval/lr_datagen.md b/docs/gitbook/eval/lr_datagen.md
index 55cb360..8fa5239 100644
--- a/docs/gitbook/eval/lr_datagen.md
+++ b/docs/gitbook/eval/lr_datagen.md
@@ -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.
+-->
+        
 _Note this feature is supported on hivemall v0.2-alpha3 or later._
 
 # create a dual table

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/fc97a52e/docs/gitbook/eval/stat_eval.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/eval/stat_eval.md b/docs/gitbook/eval/stat_eval.md
index 7f1688b..6b0af8e 100644
--- a/docs/gitbook/eval/stat_eval.md
+++ b/docs/gitbook/eval/stat_eval.md
@@ -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.
+-->
+        
 Using the [E2006 tfidf regression example](https://github.com/myui/hivemall/wiki/E2006-tfidf-regression-evaluation-(PA,-AROW)), we explain how to evaluate the prediction model on Hive.
 
 # Scoring by evaluation metrics

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/fc97a52e/docs/gitbook/ft_engineering/ft_trans.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/ft_engineering/ft_trans.md b/docs/gitbook/ft_engineering/ft_trans.md
index e69de29..2959148 100644
--- a/docs/gitbook/ft_engineering/ft_trans.md
+++ b/docs/gitbook/ft_engineering/ft_trans.md
@@ -0,0 +1,19 @@
+<!--
+  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.
+-->
+        

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/fc97a52e/docs/gitbook/ft_engineering/hashing.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/ft_engineering/hashing.md b/docs/gitbook/ft_engineering/hashing.md
index 09fa1ff..daf4a23 100644
--- a/docs/gitbook/ft_engineering/hashing.md
+++ b/docs/gitbook/ft_engineering/hashing.md
@@ -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.
+-->
+        
 Hivemall supports [Feature Hashing](https://github.com/myui/hivemall/wiki/Feature-hashing) (a.k.a. hashing trick) through `feature_hashing` and `mhash` functions. 
 Find the differences in the following examples.
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/fc97a52e/docs/gitbook/ft_engineering/quantify.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/ft_engineering/quantify.md b/docs/gitbook/ft_engineering/quantify.md
index 1d6a223..952db53 100644
--- a/docs/gitbook/ft_engineering/quantify.md
+++ b/docs/gitbook/ft_engineering/quantify.md
@@ -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.
+-->
+        
 `quantified_features` is useful for transforming values of non-number columns to indexed numbers.
 
 *Note: The feature is supported Hivemall v0.4 or later.*

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/fc97a52e/docs/gitbook/ft_engineering/scaling.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/ft_engineering/scaling.md b/docs/gitbook/ft_engineering/scaling.md
index 6e7d312..26d82bd 100644
--- a/docs/gitbook/ft_engineering/scaling.md
+++ b/docs/gitbook/ft_engineering/scaling.md
@@ -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.
+-->
+        
 # Min-Max Normalization
 http://en.wikipedia.org/wiki/Feature_scaling#Rescaling
 ```sql

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/fc97a52e/docs/gitbook/ft_engineering/tfidf.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/ft_engineering/tfidf.md b/docs/gitbook/ft_engineering/tfidf.md
index e881e10..46e4fac 100644
--- a/docs/gitbook/ft_engineering/tfidf.md
+++ b/docs/gitbook/ft_engineering/tfidf.md
@@ -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.
+-->
+        
 This document explains how to compute [TF-IDF](http://en.wikipedia.org/wiki/Tf%E2%80%93idf) with Apache Hive/Hivemall.
 
 What you need to compute TF-IDF is a table/view composing (docid, word) pair, 2 views, and 1 query.

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/fc97a52e/docs/gitbook/ft_engineering/vectorizer.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/ft_engineering/vectorizer.md b/docs/gitbook/ft_engineering/vectorizer.md
index bc929a5..59038d1 100644
--- a/docs/gitbook/ft_engineering/vectorizer.md
+++ b/docs/gitbook/ft_engineering/vectorizer.md
@@ -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.
+-->
+        
 ## Feature Vectorizer
 
 `array<string> vectorize_feature(array<string> featureNames, ...)` is useful to generate a feature vector for each row, from a table.

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/fc97a52e/docs/gitbook/getting_started/README.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/getting_started/README.md b/docs/gitbook/getting_started/README.md
index 27870e5..98393aa 100644
--- a/docs/gitbook/getting_started/README.md
+++ b/docs/gitbook/getting_started/README.md
@@ -1 +1,20 @@
+<!--
+  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.
+-->
+        
 # Summary
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/fc97a52e/docs/gitbook/getting_started/input-format.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/getting_started/input-format.md b/docs/gitbook/getting_started/input-format.md
index 272d3eb..698c095 100644
--- a/docs/gitbook/getting_started/input-format.md
+++ b/docs/gitbook/getting_started/input-format.md
@@ -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.
+-->
+        
 This page explains the input format of training data in Hivemall. 
 Here, we use [EBNF](http://en.wikipedia.org/wiki/Extended_Backus%E2%80%93Naur_Form)-like notation for describing the format.
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/fc97a52e/docs/gitbook/getting_started/installation.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/getting_started/installation.md b/docs/gitbook/getting_started/installation.md
index bb1920e..3a3c97f 100644
--- a/docs/gitbook/getting_started/installation.md
+++ b/docs/gitbook/getting_started/installation.md
@@ -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.
+-->
+        
 Prerequisites
 ============
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/fc97a52e/docs/gitbook/getting_started/permanent-functions.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/getting_started/permanent-functions.md b/docs/gitbook/getting_started/permanent-functions.md
index aab399b..a4879c3 100644
--- a/docs/gitbook/getting_started/permanent-functions.md
+++ b/docs/gitbook/getting_started/permanent-functions.md
@@ -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.
+-->
+        
 Hive v0.13 or later supports [permanent functions](https://cwiki.apache.org/confluence/display/Hive/LanguageManual+DDL#LanguageManualDDL-Create/DropFunction) that live across sessions.
 
 Permanent functions are useful when you are using Hive through Hiveserver or to avoid hivemall installation for each session.

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/fc97a52e/docs/gitbook/misc/generic_funcs.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/misc/generic_funcs.md b/docs/gitbook/misc/generic_funcs.md
index 1769699..9749dae 100644
--- a/docs/gitbook/misc/generic_funcs.md
+++ b/docs/gitbook/misc/generic_funcs.md
@@ -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.
+-->
+        
 This page describes a list of useful Hivemall generic functions.
 
 # Array functions

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/fc97a52e/docs/gitbook/misc/tokenizer.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/misc/tokenizer.md b/docs/gitbook/misc/tokenizer.md
index cd2ce08..47f07e0 100644
--- a/docs/gitbook/misc/tokenizer.md
+++ b/docs/gitbook/misc/tokenizer.md
@@ -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.
+-->
+        
 # Tokenizer for English Texts
 
 Hivemall provides simple English text tokenizer UDF that has following syntax:

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/fc97a52e/docs/gitbook/misc/topk.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/misc/topk.md b/docs/gitbook/misc/topk.md
index dcd545a..3d072ed 100644
--- a/docs/gitbook/misc/topk.md
+++ b/docs/gitbook/misc/topk.md
@@ -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.
+-->
+        
 `each_top_k(int k, ANY group, double value, arg1, arg2, ..., argN)` returns a top-k records for each `group`. It returns a relation consists of `(int rank, double value, arg1, arg2, .., argN)`.
 
 This function is particularly useful for applying a similarity/distance function where the computation complexity is **O(nm)**.

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/fc97a52e/docs/gitbook/multiclass/iris.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/multiclass/iris.md b/docs/gitbook/multiclass/iris.md
index e69de29..2959148 100644
--- a/docs/gitbook/multiclass/iris.md
+++ b/docs/gitbook/multiclass/iris.md
@@ -0,0 +1,19 @@
+<!--
+  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.
+-->
+        

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/fc97a52e/docs/gitbook/multiclass/iris_dataset.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/multiclass/iris_dataset.md b/docs/gitbook/multiclass/iris_dataset.md
index 86f89ad..38a6831 100644
--- a/docs/gitbook/multiclass/iris_dataset.md
+++ b/docs/gitbook/multiclass/iris_dataset.md
@@ -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.
+-->
+        
 # Dataset prepration
 Iris Dataset: https://archive.ics.uci.edu/ml/datasets/Iris
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/fc97a52e/docs/gitbook/multiclass/iris_randomforest.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/multiclass/iris_randomforest.md b/docs/gitbook/multiclass/iris_randomforest.md
index bafa338..fd85471 100644
--- a/docs/gitbook/multiclass/iris_randomforest.md
+++ b/docs/gitbook/multiclass/iris_randomforest.md
@@ -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.
+-->
+        
 *NOTE: RandomForest is being supported from Hivemall v0.4 or later.*
 
 # Dataset

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/fc97a52e/docs/gitbook/multiclass/iris_scw.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/multiclass/iris_scw.md b/docs/gitbook/multiclass/iris_scw.md
index bafa338..fd85471 100644
--- a/docs/gitbook/multiclass/iris_scw.md
+++ b/docs/gitbook/multiclass/iris_scw.md
@@ -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.
+-->
+        
 *NOTE: RandomForest is being supported from Hivemall v0.4 or later.*
 
 # Dataset

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/fc97a52e/docs/gitbook/multiclass/news20.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/multiclass/news20.md b/docs/gitbook/multiclass/news20.md
index e69de29..2959148 100644
--- a/docs/gitbook/multiclass/news20.md
+++ b/docs/gitbook/multiclass/news20.md
@@ -0,0 +1,19 @@
+<!--
+  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.
+-->
+        

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/fc97a52e/docs/gitbook/multiclass/news20_dataset.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/multiclass/news20_dataset.md b/docs/gitbook/multiclass/news20_dataset.md
index 35ada12..96decec 100644
--- a/docs/gitbook/multiclass/news20_dataset.md
+++ b/docs/gitbook/multiclass/news20_dataset.md
@@ -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.
+-->
+        
 Get the news20 dataset.
 http://www.csie.ntu.edu.tw/~cjlin/libsvmtools/datasets/multiclass.html#news20
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/fc97a52e/docs/gitbook/multiclass/news20_ensemble.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/multiclass/news20_ensemble.md b/docs/gitbook/multiclass/news20_ensemble.md
index 9cfd35d..6bf1c93 100644
--- a/docs/gitbook/multiclass/news20_ensemble.md
+++ b/docs/gitbook/multiclass/news20_ensemble.md
@@ -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.
+-->
+        
 This example explains how to run ensemble learning in Hivemall.   
 Two heads are better than one? Let's verify it by ensemble learning.
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/fc97a52e/docs/gitbook/multiclass/news20_one-vs-the-rest.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/multiclass/news20_one-vs-the-rest.md b/docs/gitbook/multiclass/news20_one-vs-the-rest.md
index 4c611d0..d98329f 100644
--- a/docs/gitbook/multiclass/news20_one-vs-the-rest.md
+++ b/docs/gitbook/multiclass/news20_one-vs-the-rest.md
@@ -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.
+-->
+        
 A one-vs-the-rest classifier use the binary classifier for each class.
 
 ## UDF preparation

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/fc97a52e/docs/gitbook/multiclass/news20_one-vs-the-rest_dataset.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/multiclass/news20_one-vs-the-rest_dataset.md b/docs/gitbook/multiclass/news20_one-vs-the-rest_dataset.md
index 2a69615..f437399 100644
--- a/docs/gitbook/multiclass/news20_one-vs-the-rest_dataset.md
+++ b/docs/gitbook/multiclass/news20_one-vs-the-rest_dataset.md
@@ -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.
+-->
+        
 *One-vs-the-rest* is a multiclass classification method that uses binary classifiers independently for each class.
 http://en.wikipedia.org/wiki/Multiclass_classification#one_vs_all
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/fc97a52e/docs/gitbook/multiclass/news20_pa.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/multiclass/news20_pa.md b/docs/gitbook/multiclass/news20_pa.md
index 8e69beb..26083f9 100644
--- a/docs/gitbook/multiclass/news20_pa.md
+++ b/docs/gitbook/multiclass/news20_pa.md
@@ -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.
+-->
+        
 Preparation
 =========
 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/fc97a52e/docs/gitbook/multiclass/news20_scw.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/multiclass/news20_scw.md b/docs/gitbook/multiclass/news20_scw.md
index 330c163..f6f21af 100644
--- a/docs/gitbook/multiclass/news20_scw.md
+++ b/docs/gitbook/multiclass/news20_scw.md
@@ -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.
+-->
+        
 | Algorithm | Accuracy |
 |:-----------|------------:|
 | PA2 | 0.8204357625845229 |

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/fc97a52e/docs/gitbook/recommend/cf.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/recommend/cf.md b/docs/gitbook/recommend/cf.md
index e69de29..2959148 100644
--- a/docs/gitbook/recommend/cf.md
+++ b/docs/gitbook/recommend/cf.md
@@ -0,0 +1,19 @@
+<!--
+  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.
+-->
+        

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/fc97a52e/docs/gitbook/recommend/item_based_cf.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/recommend/item_based_cf.md b/docs/gitbook/recommend/item_based_cf.md
index a4a8cfd..2eb7890 100644
--- a/docs/gitbook/recommend/item_based_cf.md
+++ b/docs/gitbook/recommend/item_based_cf.md
@@ -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.
+-->
+        
 This document describe how to do Item-based Collaborative Filtering using Hivemall.
 
 _Caution: naive similarity computation is `O(n^2)` to compute all item-item pair similarity. [MinHash](https://en.wikipedia.org/wiki/MinHash#Jaccard_similarity_and_minimum_hash_values) is an efficient scheme for computing jaccard similarity. Section 6 show how to use MinHash in Hivemall._

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/fc97a52e/docs/gitbook/recommend/movielens.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/recommend/movielens.md b/docs/gitbook/recommend/movielens.md
index e69de29..2959148 100644
--- a/docs/gitbook/recommend/movielens.md
+++ b/docs/gitbook/recommend/movielens.md
@@ -0,0 +1,19 @@
+<!--
+  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.
+-->
+        

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/fc97a52e/docs/gitbook/recommend/movielens_cv.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/recommend/movielens_cv.md b/docs/gitbook/recommend/movielens_cv.md
index ec2255b..a1f7b2f 100644
--- a/docs/gitbook/recommend/movielens_cv.md
+++ b/docs/gitbook/recommend/movielens_cv.md
@@ -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.
+-->
+        
 [Cross-validation](http://en.wikipedia.org/wiki/Cross-validation_(statistics)#k-fold_cross-validationk-fold cross validation) is a model validation technique for assessing how a prediction model will generalize to an independent data set. This example shows a way to perform [k-fold cross validation](http://en.wikipedia.org/wiki/Cross-validation_(statistics)#k-fold_cross-validation) to evaluate prediction performance.
 
 *Caution:* Matrix factorization is supported in Hivemall v0.3 or later.

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/fc97a52e/docs/gitbook/recommend/movielens_dataset.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/recommend/movielens_dataset.md b/docs/gitbook/recommend/movielens_dataset.md
index f175f4f..27c04ba 100644
--- a/docs/gitbook/recommend/movielens_dataset.md
+++ b/docs/gitbook/recommend/movielens_dataset.md
@@ -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.
+-->
+        
 # Data preparation
 
 First, downlod MovieLens dataset from the following site.

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/fc97a52e/docs/gitbook/recommend/movielens_fm.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/recommend/movielens_fm.md b/docs/gitbook/recommend/movielens_fm.md
index 282e923..eac8013 100644
--- a/docs/gitbook/recommend/movielens_fm.md
+++ b/docs/gitbook/recommend/movielens_fm.md
@@ -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.
+-->
+        
 _Caution: Factorization Machine is supported from Hivemall v0.4 or later._
 
 # Data preparation

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/fc97a52e/docs/gitbook/recommend/movielens_mf.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/recommend/movielens_mf.md b/docs/gitbook/recommend/movielens_mf.md
index bc5c641..f275df8 100644
--- a/docs/gitbook/recommend/movielens_mf.md
+++ b/docs/gitbook/recommend/movielens_mf.md
@@ -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.
+-->
+        
 This page explains how to run matrix factorization on [MovieLens 1M dataset](https://github.com/myui/hivemall/wiki/MovieLens-Dataset).
 
 *Caution:* Matrix factorization is supported in Hivemall v0.3 or later.

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/fc97a52e/docs/gitbook/recommend/news20.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/recommend/news20.md b/docs/gitbook/recommend/news20.md
index e69de29..2959148 100644
--- a/docs/gitbook/recommend/news20.md
+++ b/docs/gitbook/recommend/news20.md
@@ -0,0 +1,19 @@
+<!--
+  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.
+-->
+        

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/fc97a52e/docs/gitbook/recommend/news20_bbit_minhash.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/recommend/news20_bbit_minhash.md b/docs/gitbook/recommend/news20_bbit_minhash.md
index 72ba7f2..474a40d 100644
--- a/docs/gitbook/recommend/news20_bbit_minhash.md
+++ b/docs/gitbook/recommend/news20_bbit_minhash.md
@@ -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.
+-->
+        
 # Function Signature of bbit_minhash
 
 ```

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/fc97a52e/docs/gitbook/recommend/news20_jaccard.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/recommend/news20_jaccard.md b/docs/gitbook/recommend/news20_jaccard.md
index ea6f8cc..6a30fb8 100644
--- a/docs/gitbook/recommend/news20_jaccard.md
+++ b/docs/gitbook/recommend/news20_jaccard.md
@@ -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.
+-->
+        
 List related (similar) articles for each article.
 
 # Preparation

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/fc97a52e/docs/gitbook/recommend/news20_knn.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/recommend/news20_knn.md b/docs/gitbook/recommend/news20_knn.md
index 0471a2e..1e0ae97 100644
--- a/docs/gitbook/recommend/news20_knn.md
+++ b/docs/gitbook/recommend/news20_knn.md
@@ -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.
+-->
+        
 # Extract clusters and assign N cluster IDs to each article
 ```
 create or replace view news20_cluster

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/fc97a52e/docs/gitbook/regression/e2006.md
----------------------------------------------------------------------
diff --git a/docs/gitbook/regression/e2006.md b/docs/gitbook/regression/e2006.md
index e69de29..2959148 100644
--- a/docs/gitbook/regression/e2006.md
+++ b/docs/gitbook/regression/e2006.md
@@ -0,0 +1,19 @@
+<!--
+  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.
+-->
+