merge master into akka-streams branch

Be sure to do all of the following to help us incorporate your contribution
quickly and easily:

 - [x] Make sure the commit message is formatted like:
   `[GEARPUMP-<Jira issue #>] Meaningful description of pull request`
 - [x] Make sure tests pass via `sbt clean test`.
 - [x] Make sure old documentation affected by the pull request has been updated and new documentation added for new functionality.

Author: manuzhang <owenzhang1990@gmail.com>
Author: huafengw <fvunicorn@gmail.com>
Author: Kam Kasravi <kamkasravi@yahoo.com>
Author: darionyaphet <darion.yaphet@gmail.com>
Author: Roshanson <736781877@qq.com>
Author: Willem Jiang <willem.jiang@gmail.com>
Author: Christoph Seifert <christoph.seifert27@googlemail.com>
Author: ganesh@thedatateam.in <ganesh@thedatateam.in>

Closes #136 from manuzhang/akka-streams.
diff --git a/.github/PULL_REQUEST_TEMPLATE.md b/.github/PULL_REQUEST_TEMPLATE.md
new file mode 100644
index 0000000..175c743
--- /dev/null
+++ b/.github/PULL_REQUEST_TEMPLATE.md
@@ -0,0 +1,8 @@
+Be sure to do all of the following to help us incorporate your contribution
+quickly and easily:
+
+ - [ ] Make sure the commit message is formatted like:
+   `[GEARPUMP-<Jira issue #>] Meaningful description of pull request` 
+ - [ ] Make sure tests pass via `sbt clean test`.
+ - [ ] Make sure old documentation affected by the pull request has been updated and new documentation added for new functionality. 
+
diff --git a/.travis.yml b/.travis.yml
index e7e8abd..e47393c 100644
--- a/.travis.yml
+++ b/.travis.yml
@@ -1,12 +1,22 @@
 language:
 - java
 - scala
-sudo: false
-before_script:
-- rm -rf $HOME/.m2
-- rm -rf $HOME/.ivy2
-- mkdir -p $HOME/.sbt/launchers/0.13.11/
-- curl -L -o $HOME/.sbt/launchers/0.13.11/sbt-launch.jar http://dl.bintray.com/typesafe/ivy-releases/org.scala-sbt/sbt-launch/0.13.11/sbt-launch.jar
+sudo: required
+jdk:
+- oraclejdk8
+addons:
+  apt:
+    packages:
+      - oracle-java8-installer
+scala:
+- 2.11.8
+cache:
+  directories:
+  - $HOME/.m2/repository
+  - $HOME/.sbt
+  - $HOME/.ivy2
+git:
+  depth: 1
 script:
 - echo "TRAVIS_PULL_REQUEST" $TRAVIS_PULL_REQUEST
 - echo "TRAVIS_BRANCH" $TRAVIS_BRANCH
@@ -19,21 +29,7 @@
     then sbt -jvm-opts project/travis/jvmopts clean +assembly +publish | grep -v -E "$skipLogs"; sbt -jvm-opts project/travis/jvmopts scalastyle test:scalastyle it:scalastyle unidoc coverage  +test  | grep -v -E "$skipLogs";
   elif [[ $TRAVIS_TAG != "" ]];
     then sbt -jvm-opts project/travis/jvmopts clean +assembly +packArchiveZip | grep -v -E "$skipLogs";
-  fi  
-jdk:
-- oraclejdk8
-addons:
-  apt:
-    packages:
-      - oracle-java8-installer
-scala:
-- 2.11.8
-cache:
-  directories:
-  - $HOME/.m2
-  - $HOME/.ivy2
-git:
-  depth: 1
+  fi
 after_success:
 - if [[ $TRAVIS_PULL_REQUEST != "false" || $TRAVIS_REPO_SLUG != "apache/incubator-gearpump" || $TRAVIS_BRANCH == "master" ]];
     then bash <(curl -s https://codecov.io/bash);  
diff --git a/CHANGELOG.md b/CHANGELOG.md
index 564899e..f2522ac 100644
--- a/CHANGELOG.md
+++ b/CHANGELOG.md
@@ -1,3 +1,40 @@
+Apache Gearpump 0.8.2
+==================
+Change log:
+------------------
+- GEARPUMP-241 Update CHANGELOG.md to reflect all JIRA's 
+- GEARPUMP-240 Correct licenses
+- GEARPUMP-239 Correct binary and source licenses subdirectory
+- GEARPUMP-238 Correct release naming and packaging
+- GEARPUMP-237 do not manually download sbt launcher jar on travis
+- GEARPUMP-234 Update LICENSE and related files for 0.8.2-incubating binary release
+- GEARPUMP-230 Removed unused code from AppMasterService
+- GEARPUMP-227 remove unneeded license files
+- GEARPUMP-226 refactor build files
+- GEARPUMP-225 move partitioner from gearpump-core to module gearpump-streaming
+- GEARPUMP-224 Merge gearpump-daemon to gearpump-core
+- GEARPUMP-222 DFSJarStore is loaded even when local file path is configured
+- GEARPUMP-218 add shaded library as transitive dependencies
+- GEARPUMP-215 Gearpump Redis Integration - RedisStorage
+- GEARPUMP-213 build docs with MkDocs
+- GEARPUMP-212 Make Gearpump's shaded artifacts publishable
+- GEARPUMP-208 Demo site goes down a lot. Also references gearpump.io when connection is refused
+- GEARPUMP-207 remove hadoop dependency out of application path from dashboard
+- GEARPUMP-205 remove hdfs dependency from command gear's classpath
+- GEARPUMP-204 Add unit test for external_hbase module
+- GEARPUMP-203 Use DataSourceTask / DataSinkTask for DSL
+- GEARPUMP-201 integration test failure
+- GEARPUMP-197 Busy loop in FetchThread when incoming queue is full
+- GEARPUMP-189 Update project/Build.scala to enable publishing releases to https://repository.apache.org/content/repositories/releases/org/apache/gearpump/
+- GEARPUMP-188 use java.time.Instant for Task start time
+- GEARPUMP-185 Yarn kills Apache Gearpump's worker if the memory usage is too high
+- GEARPUMP-171 Fix some typos
+- GEARPUMP-48 Improve Java interoperability
+- GEARPUMP-37 Sometimes application clock is "not started" when the data stream is very slow
+- GEARPUMP-32 Minimum clock of source Tasks maybe inaccurate
+- GEARPUMP-23 add window DSL
+- GEARPUMP-15 Define initial Apache branded home website
+
 Apache Gearpump 0.8.1
 ==================
 Change log:
diff --git a/CONTRIBUTING.md b/CONTRIBUTING.md
index 9fd3859..a769f6d 100644
--- a/CONTRIBUTING.md
+++ b/CONTRIBUTING.md
@@ -267,7 +267,7 @@
 <a name="ide-setup"></a>
 ## IDE setup
 IDE environment can be set up on either Windows, Linux and Mac platform. You can choose the one you prefer. 
-The IDE setup guide can be found at [Gearpump website](http://gearpump.apache.org/releases/latest/dev-ide-setup.html).
+The IDE setup guide can be found at [Gearpump website](http://gearpump.apache.org/releases/latest/dev/dev-ide-setup/index.html).
 
 It is highly recommended to perform [package build](#build) before IDE setup.
 
diff --git a/LICENSE b/LICENSE
index 5bc72ee..7e555a5 100644
--- a/LICENSE
+++ b/LICENSE
@@ -209,11 +209,6 @@
    code for these subcomponents is subject to the terms and
    conditions of the following licenses. 
 
-
-   This product bundles AnchorJS 1.1.1, copyright Bryan Braun,
-   which is available under a MIT license.
-   For details see licenses/LICENSE-AnchorJS.txt
-
    This product bundles Bootstrap 3.3.5, copyright Twitter Inc.,
    which is available under a MIT license.
    For details see licenses/LICENSE-bootstrap.txt
@@ -221,23 +216,6 @@
    This product bundles jquery 2.1.4, copyright jQuery Foundation, Inc.,
    which is available under a MIT license. 
    For details see licenses/LICENSE-jquery.txt
-   
-   This product bundles modernizr 2.6.1, copyright Faruk Ates, Paul Irish,
-   Alex Sexton, Ryan Seddon, Patrick Kettner, Stu Cox and Richard Herrera,
-   which is available under a MIT license.
-   For details see licenses/LICENSE-modernizr.txt
-
-   This product bundles normalize 3.0.3, copyright Nicolas Gallagher and
-   Jonathan Neal, which is available under a MIT license.
-   For details, see licenses/normalize.MIT.
-
-   This product bundles Glyphicons Halflings Regular, copyright
-   Jan Kovarik, which is available under a MIT license.
-   For details, see licenses/glyphicons.MIT.
-
-   This product bundles polyfill, copyright Scott Jehl, Paul Irish and
-   Nicholas Zakas, which is available under a MIT license.
-   For details, see licenses/glyphicons.MIT.
 
 
 
diff --git a/LICENSE.bin b/LICENSE.bin
new file mode 100644
index 0000000..52b09fe
--- /dev/null
+++ b/LICENSE.bin
@@ -0,0 +1,344 @@
+                                 Apache License
+                           Version 2.0, January 2004
+                        http://www.apache.org/licenses/
+
+   TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION
+
+   1. Definitions.
+
+      "License" shall mean the terms and conditions for use, reproduction,
+      and distribution as defined by Sections 1 through 9 of this document.
+
+      "Licensor" shall mean the copyright owner or entity authorized by
+      the copyright owner that is granting the License.
+
+      "Legal Entity" shall mean the union of the acting entity and all
+      other entities that control, are controlled by, or are under common
+      control with that entity. For the purposes of this definition,
+      "control" means (i) the power, direct or indirect, to cause the
+      direction or management of such entity, whether by contract or
+      otherwise, or (ii) ownership of fifty percent (50%) or more of the
+      outstanding shares, or (iii) beneficial ownership of such entity.
+
+      "You" (or "Your") shall mean an individual or Legal Entity
+      exercising permissions granted by this License.
+
+      "Source" form shall mean the preferred form for making modifications,
+      including but not limited to software source code, documentation
+      source, and configuration files.
+
+      "Object" form shall mean any form resulting from mechanical
+      transformation or translation of a Source form, including but
+      not limited to compiled object code, generated documentation,
+      and conversions to other media types.
+
+      "Work" shall mean the work of authorship, whether in Source or
+      Object form, made available under the License, as indicated by a
+      copyright notice that is included in or attached to the work
+      (an example is provided in the Appendix below).
+
+      "Derivative Works" shall mean any work, whether in Source or Object
+      form, that is based on (or derived from) the Work and for which the
+      editorial revisions, annotations, elaborations, or other modifications
+      represent, as a whole, an original work of authorship. For the purposes
+      of this License, Derivative Works shall not include works that remain
+      separable from, or merely link (or bind by name) to the interfaces of,
+      the Work and Derivative Works thereof.
+
+      "Contribution" shall mean any work of authorship, including
+      the original version of the Work and any modifications or additions
+      to that Work or Derivative Works thereof, that is intentionally
+      submitted to Licensor for inclusion in the Work by the copyright owner
+      or by an individual or Legal Entity authorized to submit on behalf of
+      the copyright owner. For the purposes of this definition, "submitted"
+      means any form of electronic, verbal, or written communication sent
+      to the Licensor or its representatives, including but not limited to
+      communication on electronic mailing lists, source code control systems,
+      and issue tracking systems that are managed by, or on behalf of, the
+      Licensor for the purpose of discussing and improving the Work, but
+      excluding communication that is conspicuously marked or otherwise
+      designated in writing by the copyright owner as "Not a Contribution."
+
+      "Contributor" shall mean Licensor and any individual or Legal Entity
+      on behalf of whom a Contribution has been received by Licensor and
+      subsequently incorporated within the Work.
+
+   2. Grant of Copyright License. Subject to the terms and conditions of
+      this License, each Contributor hereby grants to You a perpetual,
+      worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+      copyright license to reproduce, prepare Derivative Works of,
+      publicly display, publicly perform, sublicense, and distribute the
+      Work and such Derivative Works in Source or Object form.
+
+   3. Grant of Patent License. Subject to the terms and conditions of
+      this License, each Contributor hereby grants to You a perpetual,
+      worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+      (except as stated in this section) patent license to make, have made,
+      use, offer to sell, sell, import, and otherwise transfer the Work,
+      where such license applies only to those patent claims licensable
+      by such Contributor that are necessarily infringed by their
+      Contribution(s) alone or by combination of their Contribution(s)
+      with the Work to which such Contribution(s) was submitted. If You
+      institute patent litigation against any entity (including a
+      cross-claim or counterclaim in a lawsuit) alleging that the Work
+      or a Contribution incorporated within the Work constitutes direct
+      or contributory patent infringement, then any patent licenses
+      granted to You under this License for that Work shall terminate
+      as of the date such litigation is filed.
+
+   4. Redistribution. You may reproduce and distribute copies of the
+      Work or Derivative Works thereof in any medium, with or without
+      modifications, and in Source or Object form, provided that You
+      meet the following conditions:
+
+      (a) You must give any other recipients of the Work or
+          Derivative Works a copy of this License; and
+
+      (b) You must cause any modified files to carry prominent notices
+          stating that You changed the files; and
+
+      (c) You must retain, in the Source form of any Derivative Works
+          that You distribute, all copyright, patent, trademark, and
+          attribution notices from the Source form of the Work,
+          excluding those notices that do not pertain to any part of
+          the Derivative Works; and
+
+      (d) If the Work includes a "NOTICE" text file as part of its
+          distribution, then any Derivative Works that You distribute must
+          include a readable copy of the attribution notices contained
+          within such NOTICE file, excluding those notices that do not
+          pertain to any part of the Derivative Works, in at least one
+          of the following places: within a NOTICE text file distributed
+          as part of the Derivative Works; within the Source form or
+          documentation, if provided along with the Derivative Works; or,
+          within a display generated by the Derivative Works, if and
+          wherever such third-party notices normally appear. The contents
+          of the NOTICE file are for informational purposes only and
+          do not modify the License. You may add Your own attribution
+          notices within Derivative Works that You distribute, alongside
+          or as an addendum to the NOTICE text from the Work, provided
+          that such additional attribution notices cannot be construed
+          as modifying the License.
+
+      You may add Your own copyright statement to Your modifications and
+      may provide additional or different license terms and conditions
+      for use, reproduction, or distribution of Your modifications, or
+      for any such Derivative Works as a whole, provided Your use,
+      reproduction, and distribution of the Work otherwise complies with
+      the conditions stated in this License.
+
+   5. Submission of Contributions. Unless You explicitly state otherwise,
+      any Contribution intentionally submitted for inclusion in the Work
+      by You to the Licensor shall be under the terms and conditions of
+      this License, without any additional terms or conditions.
+      Notwithstanding the above, nothing herein shall supersede or modify
+      the terms of any separate license agreement you may have executed
+      with Licensor regarding such Contributions.
+
+   6. Trademarks. This License does not grant permission to use the trade
+      names, trademarks, service marks, or product names of the Licensor,
+      except as required for reasonable and customary use in describing the
+      origin of the Work and reproducing the content of the NOTICE file.
+
+   7. Disclaimer of Warranty. Unless required by applicable law or
+      agreed to in writing, Licensor provides the Work (and each
+      Contributor provides its Contributions) on an "AS IS" BASIS,
+      WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
+      implied, including, without limitation, any warranties or conditions
+      of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A
+      PARTICULAR PURPOSE. You are solely responsible for determining the
+      appropriateness of using or redistributing the Work and assume any
+      risks associated with Your exercise of permissions under this License.
+
+   8. Limitation of Liability. In no event and under no legal theory,
+      whether in tort (including negligence), contract, or otherwise,
+      unless required by applicable law (such as deliberate and grossly
+      negligent acts) or agreed to in writing, shall any Contributor be
+      liable to You for damages, including any direct, indirect, special,
+      incidental, or consequential damages of any character arising as a
+      result of this License or out of the use or inability to use the
+      Work (including but not limited to damages for loss of goodwill,
+      work stoppage, computer failure or malfunction, or any and all
+      other commercial damages or losses), even if such Contributor
+      has been advised of the possibility of such damages.
+
+   9. Accepting Warranty or Additional Liability. While redistributing
+      the Work or Derivative Works thereof, You may choose to offer,
+      and charge a fee for, acceptance of support, warranty, indemnity,
+      or other liability obligations and/or rights consistent with this
+      License. However, in accepting such obligations, You may act only
+      on Your own behalf and on Your sole responsibility, not on behalf
+      of any other Contributor, and only if You agree to indemnify,
+      defend, and hold each Contributor harmless for any liability
+      incurred by, or claims asserted against, such Contributor by reason
+      of your accepting any such warranty or additional liability.
+
+   END OF TERMS AND CONDITIONS
+
+   APPENDIX: How to apply the Apache License to your work.
+
+      To apply the Apache License to your work, attach the following
+      boilerplate notice, with the fields enclosed by brackets "[]"
+      replaced with your own identifying information. (Don't include
+      the brackets!)  The text should be enclosed in the appropriate
+      comment syntax for the file format. We also recommend that a
+      file or class name and description of purpose be included on the
+      same "printed page" as the copyright notice for easier
+      identification within third-party archives.
+
+   Copyright [yyyy] [name of copyright owner]
+
+   Licensed under the Apache License, Version 2.0 (the "License");
+   you may not use this file except in compliance with the License.
+   You may obtain a copy of the License at
+
+       http://www.apache.org/licenses/LICENSE-2.0
+
+   Unless required by applicable law or agreed to in writing, software
+   distributed under the License is distributed on an "AS IS" BASIS,
+   WITHOUT 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 Gearpump subcomponents:
+
+   Apache Gearpump includes a number of subcomponents with
+   separate copyright notices and license terms. Your use of the source
+   code for these subcomponents is subject to the terms and
+   conditions of the following licenses. 
+
+========================================================================
+BSD licenses
+The text of each license is also included at licenses/LICENSE-[project].txt.
+See project link for details.
+========================================================================
+com.jcraft:jsch:0.1.42 - http://www.jcraft.com/jsch/
+com.thoughtworks.paranamer:paranamer:2.3 - https://github.com/paul-hammant/paranamer
+jline:jline:0.9.94 - http://jline.sourceforge.net/
+net.sourceforge.htmlcleaner:htmlcleaner:2.2 - http://htmlcleaner.sourceforge.net/download.php
+asm.asm:3.1 - http://asm.ow2.org/
+org.ow2.asm:asm:4.0 - http://asm.ow2.org/
+org.ow2.asm:asm:5.0.3 - http://asm.ow2.org/
+org.scala-stm:scala-stm_2.11:0.7 - https://nbronson.github.io/scala-stm/releases.html
+xmlenc:xmlenc:0.52 - http://xmlenc.sourceforge.net/
+
+========================================================================
+BSD 3-clause licenses
+The text of each license is also included at licenses/LICENSE-[project].txt.
+See project link for details.
+========================================================================
+org.scala-lang:scala-reflect:2.11.8 - http://www.scala-lang.org/download/#License
+org.scala-lang.modules:scala-java8-compat_2.11:0.7.0 - http://www.scala-lang.org/download/#License
+org.scala-lang.modules:scala-parser-combinators_2.11:1.0.2 - http://www.scala-lang.org/download/#License
+org.scala-lang.modules:scala-parser-combinators_2.11:1.0.4 - http://www.scala-lang.org/download/#License
+org.scala-lang.modules:scala-xml_2.11:1.0.2 - http://www.scala-lang.org/download/#License
+org.webjars:d3js:3.5.6 - https://d3js.org/
+
+========================================================================
+New BSD licenses
+The text of each license is also included at licenses/LICENSE-[project].txt.
+See project link for details.
+========================================================================
+com.esotericsoftware.kryo:kryo:2.21 - https://github.com/EsotericSoftware/kryo/
+com.esotericsoftware.minlog:minlog:1.2 - https://github.com/EsotericSoftware/minlog
+com.esotericsoftware.reflectasm:reflectasm:1.07 - https://github.com/EsotericSoftware/reflectasm
+com.esotericsoftware:kryo:3.0.3 - https://github.com/EsotericSoftware/kryo/
+com.esotericsoftware:minlog:1.3.0 - https://github.com/EsotericSoftware/minlog
+com.esotericsoftware:reflectasm:1.10.1 - https://github.com/EsotericSoftware/reflectasm
+com.google.protobuf:protobuf-java:2.5.0 - https://developers.google.com/protocol-buffers/
+
+========================================================================
+CC BY 3.0 license
+The text of each license is also included at licenses/LICENSE-[project].txt.
+See project link for details.
+========================================================================
+org.webjars:font-awesome:4.5.0 - http://fontawesome.io/
+ 
+========================================================================
+CC0 license
+The text of each license is also included at licenses/LICENSE-[project].txt.
+See project link for details.
+========================================================================
+org.reactivestreams:reactive-streams:1.0.0 - http://www.reactive-streams.org/
+
+========================================================================
+CDDL 1.1 license
+The text of each license is also included at licenses/LICENSE-[project].txt.
+See project link for details.
+========================================================================
+com.sun.jersey.contribs:jersey-guice:1.9 - https://jersey.java.net/
+com.sun.jersey:jersey-client:1.9 - https://jersey.java.net/
+com.sun.jersey:jersey-core:1.9 - https://jersey.java.net/
+com.sun.jersey:jersey-json:1.9 - https://jersey.java.net/
+com.sun.jersey:jersey-server:1.9 - https://jersey.java.net/
+com.sun.xml.bind:jaxb-impl:2.2.3-1 - https://jaxb.java.net/
+javax.xml.bind:jaxb-api:2.2.2 - https://jaxb.java.net/
+org.jvnet.mimepull:mimepull:1.9.4 - https://mimepull.java.net/
+
+========================================================================
+Common Development and Distribution License (CDDL) v1.0
+The text of each license is also included at licenses/LICENSE-[project].txt.
+See project link for details.
+========================================================================
+javax.activation:activation:1.1 - http://www.oracle.com/technetwork/java/index.html
+javax.servlet:servlet-api:2.5 - http://www.oracle.com/technetwork/java/index.html
+javax.xml.stream:stax-api:1.0-2 - https://sjsxp.java.net/
+javax.servlet.jsp:jsp-api:2.1 - http://www.oracle.com/technetwork/java/index.html
+
+========================================================================
+Common Public License Version 1.0
+The text of each license is also included at licenses/LICENSE-[project].txt.
+See project link for details.
+========================================================================
+junit:junit:3.8.1 - http://junit.org
+
+========================================================================
+Eclipse Public License 1.0
+The text of each license is also included at licenses/LICENSE-[project].txt.
+See project link for details.
+========================================================================
+org.clojure:clojure:1.6.0 - http://clojure.org/
+ 
+========================================================================
+MIT licenses
+The text of each license is also included at licenses/LICENSE-[project].txt.
+See project link for details.
+========================================================================
+com.github.scribejava:scribejava-apis:2.4.0 - https://github.com/scribejava/scribejava
+com.github.scribejava:scribejava-core:2.4.0 - https://github.com/scribejava/scribejava
+com.lihaoyi:derive_2.11:0.3.4 - https://github.com/lihaoyi/upickle
+com.lihaoyi:upickle_2.11:0.3.4 - https://github.com/lihaoyi/upickle
+org.jruby.jcodings:jcodings:1.0.8 - https://github.com/jruby/jcodings
+org.jruby.joni:joni:2.1.2 - https://github.com/jruby/joni
+org.slf4j:slf4j-api:1.7.16 - http://www.slf4j.org/
+org.slf4j:slf4j-api:1.7.12 - http://www.slf4j.org/
+org.slf4j:slf4j-log4j12:1.7.16 - http://www.slf4j.org/
+org.spire-math:jawn-parser_2.11:0.7.0 - https://github.com/non/jawn/
+org.webjars:angular-ui-router:0.2.15 - https://github.com/angular-ui/ui-router
+org.webjars:angularjs:1.4.9 - https://angularjs.org
+org.webjars.bower:angular-loading-bar:0.8.0 - https://chieffancypants.github.io/angular-loading-bar/
+org.webjars.bower:angular-motion:0.4.3 - http://www.webjars.org/
+org.webjars.bower:angular-smart-table:2.1.6 - https://lorenzofox3.github.io/smart-table-website/
+org.webjars.bower:angular-strap:2.3.5 - http://mgcrea.github.io/angular-strap/
+org.webjars.bower:bootstrap:0.3.3.5 = https://getbootstrap.com/
+org.webjars.bower:bootstrap-additions:0.3.1 = https://github.com/mgcrea/bootstrap-additions
+org.webjars.bower:clipboard.js:0.1.1 - https://clipboardjs.com/
+org.webjars.bower:ng-file-upload:5.0.9 - https://github.com/danialfarid/ng-file-upload
+org.webjars:lodash:3.10.1 - https://lodash.com/
+org.webjars:jquery:2.2.0 - https://jquery.com/
+org.webjars:jquery-cookie:1.4.1 - https://github.com/js-cookie/js-cookie
+org.webjars:momentjs:2.10.6 - http://momentjs.com/
+org.webjars.npm:angular-touch:1.5.0 - http://angularjs.org
+org.webjars.npm:ui-select:0.14.2 - http://angular-ui.github.io/ui-select/
+redis.clients:jedis:2.9.0 - https://github.com/xetorthio/jedis
+net.sf.jopt-simple:jopt-simple:3.2 - https://pholser.github.io/jopt-simple/
+
+========================================================================
+Public Domain
+The text of each license is also included at licenses/LICENSE-[project].txt.
+See project link for details.
+========================================================================
+[info] 	 aopalliance:aopalliance:1.0 - http://aopalliance.sourceforge.net/
+[info] 	 org.tukaani:xz:1.0 - http://tukaani.org/xz/java.html
diff --git a/NOTICE b/NOTICE
index da0bf4f..ca86fe0 100644
--- a/NOTICE
+++ b/NOTICE
@@ -1,14 +1,5 @@
-Apache Gearpump
+Apache Gearpump (incubating)
 Copyright 2015-2016 The Apache Software Foundation 
    
 This product includes software developed at
 The Apache Software Foundation (http://www.apache.org/).
-
-=======================================================================
-
-Apache Gearpump contains subcomponents with separate copyright notices and
-license terms. Your use of the source code for the these subcomponents
-is subject to the terms and conditions of their respective licenses.
-
-See the LICENSE file for a list of subcomponents and dependencies and
-their respective licenses.
diff --git a/README.md b/README.md
index 7258155..9edd0f7 100644
--- a/README.md
+++ b/README.md
@@ -1,12 +1,8 @@
 
-## Apache Gearpump (incubating)  [![GitHub release](https://img.shields.io/github/release/gearpump/gearpump.svg)](http://gearpump.apache.org/downloads.html) [![GitHub license](https://img.shields.io/badge/license-Apache%20V2-green.svg)](https://github.com/apache/incubator-gearpump/blob/master/LICENSE) [![Build Status](https://travis-ci.org/apache/incubator-gearpump.svg?branch=master)](https://travis-ci.org/apache/incubator-gearpump?branch=master) [![codecov.io](https://codecov.io/github/apache/incubator-gearpump/coverage.svg?branch=master)](https://codecov.io/github/apache/incubator-gearpump?branch=master)
-
-[![download](https://raw.githubusercontent.com/clockfly/icons/master/gearpump-0.2-rc1.jpg)](http://gearpump.apache.org/downloads.html)
+## Apache Gearpump (incubating)  [![Maven Central](https://maven-badges.herokuapp.com/maven-central/org.apache.gearpump/gearpump-core_2.11/badge.svg)](http://gearpump.apache.org/downloads.html) [![GitHub license](https://img.shields.io/badge/license-Apache%20V2-green.svg)](https://github.com/apache/incubator-gearpump/blob/master/LICENSE) [![Build Status](https://travis-ci.org/apache/incubator-gearpump.svg?branch=master)](https://travis-ci.org/apache/incubator-gearpump?branch=master) [![codecov.io](https://codecov.io/github/apache/incubator-gearpump/coverage.svg?branch=master)](https://codecov.io/github/apache/incubator-gearpump?branch=master)
 
 Gearpump is a lightweight real-time big data streaming engine. It is inspired by recent advances in the [Akka](https://github.com/akka/akka) framework and a desire to improve on existing streaming frameworks.
 
-![](https://raw.githubusercontent.com/apache/incubator-gearpump/master/docs/docs/img/logo.png)
-
 The name Gearpump is a reference to the engineering term "gear pump", which is a super simple pump that consists of only two gears, but is very powerful at streaming water.
 
 ![](http://gearpump.apache.org/img/dashboard.gif)
diff --git a/ReleaseProcess.md b/ReleaseProcess.md
index 4ecf460..e1b188a 100644
--- a/ReleaseProcess.md
+++ b/ReleaseProcess.md
@@ -11,17 +11,32 @@
 
 Step1: Pre-release
 ===================
-1. Modify CHANGELOG.md to add JIRA's
-2. Update version in docs/_config.yml
+1. Modify CHANGELOG.md to add JIRA's for the release
+2. Update version in docs/version.yml
 3. Bump the gearpump version in version.sbt 
-
    ```scala
    version in ThisBuild := "RELEASE_VERSION"
    ```
-  
-4. Run dev-tools/create_apache_source_release.sh $GPG_KEY $GPG_PASSPHRASE
-5. svn add gearpump-* to https://dist.apache.org/repos/dist/dev/incubator/gearpump/RELEASE_VERSION-incubating/RC[0-9]
-6. svn add KEYS to https://dist.apache.org/repos/dist/dev/incubator/gearpump/
+4. Run dev-tools/dependencies.sh
+   This will generate a LICENSE.dependencies file that lists all dependencies including Apache.
+   Make sure this agrees with the LICENSE and license/* files.
+   Eventually we'll have something like a verify option so the inspection isn't manual.
+5. Run dev-tools/create_apache_source_release.sh $GPG_KEY $GPG_PASSPHRASE
+   This will provide the source artifacts that need to be uploaded in step 6. below
+6. Upload to svn 
+   Run 'svn checkout https://dist.apache.org/repos/dist/dev/incubator/gearpump'
+   Run 'svn mkdir RELEASE_VERSION-incubating'
+   Run 'svn mkdir RELEASE_VERSION-incubating/RC[0-9]'
+   cp the gearpump* files generated from 5. to RELEASE_VERSION-incubating/RC[0-9]
+   Run 'svn add RELEASE_VERSION-incubating/RC[0-9]/gearpump*'
+   Run 'svn commit'
+7. Run dev-tools/create_apache_bin_release.sh $GPG_KEY $GPG_PASSPHRASE
+   This will provide the binary artifacts that need to be uploaded in step 8. below
+8. svn add gearpump-* to https://dist.apache.org/repos/dist/dev/incubator/gearpump/RELEASE_VERSION-incubating/RC[0-9]
+9. svn add KEYS to https://dist.apache.org/repos/dist/dev/incubator/gearpump/
+   This only needs to be done if we are adding new committers for this release
+10.Create a tag for the RC release by ```git tag RELEASE_VERION-RC[0-9]```
+11.Push this tag upstream and merge
 
 Step2: Release
 ==================
diff --git a/conf/gear.conf b/conf/gear.conf
index fa0a3e9..52c67b8 100644
--- a/conf/gear.conf
+++ b/conf/gear.conf
@@ -282,6 +282,11 @@
   ### Configuration for yarn module
   ###########################
   yarn {
+    ## By default the owner of yarn application(Here is a Gearpump cluster) is the one who
+    ## launches yarnclient.
+    ## Now you delegat to other users by setting following config.
+    # user = "guest"
+
     client {
       package-path = "/usr/lib/gearpump/gearpump.zip"
     }
diff --git a/core/src/main/java/org/apache/gearpump/transport/netty/MessageBatch.java b/core/src/main/java/org/apache/gearpump/transport/netty/MessageBatch.java
index d9de871..13ce9fa 100644
--- a/core/src/main/java/org/apache/gearpump/transport/netty/MessageBatch.java
+++ b/core/src/main/java/org/apache/gearpump/transport/netty/MessageBatch.java
@@ -18,7 +18,7 @@
 
 package org.apache.gearpump.transport.netty;
 
-import org.apache.gearpump.google.common.io.Closeables;
+import com.google.common.io.Closeables;
 import org.jboss.netty.buffer.ChannelBuffer;
 import org.jboss.netty.buffer.ChannelBufferOutputStream;
 import org.jboss.netty.buffer.ChannelBuffers;
diff --git a/daemon/src/main/resources/META-INF/services/org.apache.gearpump.jarstore.JarStore b/core/src/main/resources/META-INF/services/org.apache.gearpump.jarstore.JarStore
similarity index 94%
copy from daemon/src/main/resources/META-INF/services/org.apache.gearpump.jarstore.JarStore
copy to core/src/main/resources/META-INF/services/org.apache.gearpump.jarstore.JarStore
index e173a8a..c64d444 100644
--- a/daemon/src/main/resources/META-INF/services/org.apache.gearpump.jarstore.JarStore
+++ b/core/src/main/resources/META-INF/services/org.apache.gearpump.jarstore.JarStore
@@ -16,5 +16,4 @@
 # limitations under the License.
 #
 
-org.apache.gearpump.jarstore.local.LocalJarStore
-org.apache.gearpump.jarstore.dfs.DFSJarStore
\ No newline at end of file
+org.apache.gearpump.jarstore.local.LocalJarStore
\ No newline at end of file
diff --git a/daemon/src/main/scala/org/apache/gearpump/cluster/DaemonMessage.scala b/core/src/main/scala/org/apache/gearpump/cluster/DaemonMessage.scala
similarity index 99%
rename from daemon/src/main/scala/org/apache/gearpump/cluster/DaemonMessage.scala
rename to core/src/main/scala/org/apache/gearpump/cluster/DaemonMessage.scala
index 9e55be6..1e94132 100644
--- a/daemon/src/main/scala/org/apache/gearpump/cluster/DaemonMessage.scala
+++ b/core/src/main/scala/org/apache/gearpump/cluster/DaemonMessage.scala
@@ -18,7 +18,6 @@
 package org.apache.gearpump.cluster
 
 import akka.actor.ActorRef
-
 import org.apache.gearpump.cluster.master.Master.MasterInfo
 import org.apache.gearpump.cluster.scheduler.Resource
 import org.apache.gearpump.cluster.worker.WorkerId
diff --git a/core/src/main/scala/org/apache/gearpump/cluster/UserConfig.scala b/core/src/main/scala/org/apache/gearpump/cluster/UserConfig.scala
index 393d5f7..1946cfb 100644
--- a/core/src/main/scala/org/apache/gearpump/cluster/UserConfig.scala
+++ b/core/src/main/scala/org/apache/gearpump/cluster/UserConfig.scala
@@ -20,16 +20,13 @@
 
 import akka.actor.{ActorSystem, ExtendedActorSystem}
 import akka.serialization.JavaSerializer
-import org.apache.gearpump.google.common.io.BaseEncoding
-import org.apache.gearpump.util.LogUtil
+import com.google.common.io.BaseEncoding
 
-import scala.util.{Failure, Success, Try}
 
 /**
  * Immutable configuration
  */
 final class UserConfig(private val _config: Map[String, String]) extends Serializable {
-  private val LOG = LogUtil.getLogger(getClass)
 
   def withBoolean(key: String, value: Boolean): UserConfig = {
     new UserConfig(_config + (key -> value.toString))
@@ -138,18 +135,10 @@
     if (null == value) {
       this
     } else {
-      Try({
-        val serializer = new JavaSerializer(system.asInstanceOf[ExtendedActorSystem])
-        val bytes = serializer.toBinary(value)
-        BaseEncoding.base64().encode(bytes)
-      }) match {
-        case Success(enc) =>
-          this.withString(key, enc)
-        case Failure(throwable) =>
-          LOG.error(s"Could not serialize value with key $key ${throwable.getMessage}")
-          this
-      }
-
+      val serializer = new JavaSerializer(system.asInstanceOf[ExtendedActorSystem])
+      val bytes = serializer.toBinary(value)
+      val encoded = BaseEncoding.base64().encode(bytes)
+      this.withString(key, encoded)
     }
   }
   // scalastyle:on line.size.limit
diff --git a/core/src/main/scala/org/apache/gearpump/cluster/client/ClientContext.scala b/core/src/main/scala/org/apache/gearpump/cluster/client/ClientContext.scala
old mode 100644
new mode 100755
index 0cba079..48b95d8
--- a/core/src/main/scala/org/apache/gearpump/cluster/client/ClientContext.scala
+++ b/core/src/main/scala/org/apache/gearpump/cluster/client/ClientContext.scala
@@ -19,23 +19,25 @@
 package org.apache.gearpump.cluster.client
 
 import java.util.concurrent.TimeUnit
-import scala.collection.JavaConverters._
-import scala.concurrent.duration.Duration
-import scala.concurrent.{Await, Future}
-import scala.util.Try
 
 import akka.actor.{ActorRef, ActorSystem}
 import akka.util.Timeout
 import com.typesafe.config.{Config, ConfigValueFactory}
-import org.slf4j.Logger
-
-import org.apache.gearpump.cluster.MasterToAppMaster.{AppMastersData, ReplayFromTimestampWindowTrailingEdge}
-import org.apache.gearpump.cluster.MasterToClient.ReplayApplicationResult
+import org.apache.gearpump.cluster.ClientToMaster.{ResolveAppId, ShutdownApplication, SubmitApplication}
+import org.apache.gearpump.cluster.MasterToAppMaster.{AppMastersData, AppMastersDataRequest, ReplayFromTimestampWindowTrailingEdge}
+import org.apache.gearpump.cluster.MasterToClient._
 import org.apache.gearpump.cluster._
 import org.apache.gearpump.cluster.master.MasterProxy
-import org.apache.gearpump.jarstore.{JarStoreClient, JarStoreServer}
+import org.apache.gearpump.jarstore.JarStoreClient
 import org.apache.gearpump.util.Constants._
 import org.apache.gearpump.util.{ActorUtil, Constants, LogUtil, Util}
+import org.slf4j.Logger
+
+import scala.concurrent.ExecutionContext.Implicits.global
+import scala.collection.JavaConverters._
+import scala.concurrent.duration.Duration
+import scala.concurrent.{Await, Future}
+import scala.util.{Failure, Success, Try}
 
 /**
  * ClientContext is a user facing util to submit/manage an application.
@@ -43,7 +45,6 @@
  * TODO: add interface to query master here
  */
 class ClientContext(config: Config, sys: ActorSystem, _master: ActorRef) {
-
   def this(system: ActorSystem) = {
     this(system.settings.config, system, null)
   }
@@ -53,20 +54,20 @@
   }
 
   private val LOG: Logger = LogUtil.getLogger(getClass)
-  private implicit val timeout = Timeout(5, TimeUnit.SECONDS)
-
   implicit val system = Option(sys).getOrElse(ActorSystem(s"client${Util.randInt()}", config))
   LOG.info(s"Starting system ${system.name}")
-  val shouldCleanupSystem = Option(sys).isEmpty
-
   private val jarStoreClient = new JarStoreClient(config, system)
+  private val masterClientTimeout = {
+    val timeout = Try(config.getInt(Constants.GEARPUMP_MASTERCLIENT_TIMEOUT)).getOrElse(90)
+    Timeout(timeout, TimeUnit.SECONDS)
+  }
 
   private lazy val master: ActorRef = {
     val masters = config.getStringList(Constants.GEARPUMP_CLUSTER_MASTERS).asScala
       .flatMap(Util.parseHostList)
     val master = Option(_master).getOrElse(system.actorOf(MasterProxy.props(masters),
       s"masterproxy${system.name}"))
-    LOG.info(s"Creating master proxy ${master} for master list: $masters")
+    LOG.info(s"Creating master proxy $master for master list: $masters")
     master
   }
 
@@ -75,26 +76,25 @@
    * defined. Otherwise, it assumes the jar is on the target runtime classpath, thus will
    * not send the jar across the wire.
    */
-  def submit(app: Application): Int = {
+  def submit(app: Application): RunningApplication = {
     submit(app, System.getProperty(GEARPUMP_APP_JAR))
   }
 
-  def submit(app: Application, jar: String): Int = {
-    submit(app, jar, getExecutorNum())
+  def submit(app: Application, jar: String): RunningApplication = {
+    submit(app, jar, getExecutorNum)
   }
 
-  def submit(app: Application, jar: String, executorNum: Int): Int = {
-    val client = getMasterClient
+  def submit(app: Application, jar: String, executorNum: Int): RunningApplication = {
     val appName = checkAndAddNamePrefix(app.name, System.getProperty(GEARPUMP_APP_NAME_PREFIX))
     val submissionConfig = getSubmissionConfig(config)
       .withValue(APPLICATION_EXECUTOR_NUMBER, ConfigValueFactory.fromAnyRef(executorNum))
     val appDescription =
       AppDescription(appName, app.appMaster.getName, app.userConfig, submissionConfig)
     val appJar = Option(jar).map(loadFile)
-    client.submitApplication(appDescription, appJar)
+    submitApplication(SubmitApplication(appDescription, appJar))
   }
 
-  private def getExecutorNum(): Int = {
+  private def getExecutorNum: Int = {
     Try(System.getProperty(APPLICATION_EXECUTOR_NUMBER).toInt).getOrElse(1)
   }
 
@@ -102,8 +102,11 @@
     ClusterConfig.filterOutDefaultConfig(config)
   }
 
+  def listApps: AppMastersData = {
+    ActorUtil.askActor[AppMastersData](master, AppMastersDataRequest, masterClientTimeout)
+  }
+
   def replayFromTimestampWindowTrailingEdge(appId: Int): ReplayApplicationResult = {
-    import scala.concurrent.ExecutionContext.Implicits.global
     val result = Await.result(
       ActorUtil.askAppMaster[ReplayApplicationResult](master,
         appId, ReplayFromTimestampWindowTrailingEdge(appId)), Duration.Inf)
@@ -111,27 +114,29 @@
   }
 
   def askAppMaster[T](appId: Int, msg: Any): Future[T] = {
-    import scala.concurrent.ExecutionContext.Implicits.global
     ActorUtil.askAppMaster[T](master, appId, msg)
   }
 
-  def listApps: AppMastersData = {
-    val client = getMasterClient
-    client.listApplications
-  }
-
   def shutdown(appId: Int): Unit = {
-    val client = getMasterClient
-    client.shutdownApplication(appId)
+    val result = ActorUtil.askActor[ShutdownApplicationResult](master,
+      ShutdownApplication(appId), masterClientTimeout)
+    result.appId match {
+      case Success(_) =>
+      case Failure(ex) => throw ex
+    }
   }
 
   def resolveAppID(appId: Int): ActorRef = {
-    val client = getMasterClient
-    client.resolveAppId(appId)
+    val result = ActorUtil.askActor[ResolveAppIdResult](master,
+      ResolveAppId(appId), masterClientTimeout)
+    result.appMaster match {
+      case Success(appMaster) => appMaster
+      case Failure(ex) => throw ex
+    }
   }
 
   def close(): Unit = {
-    if (shouldCleanupSystem) {
+    if (sys == null) {
       LOG.info(s"Shutting down system ${system.name}")
       system.terminate()
     }
@@ -139,6 +144,10 @@
 
   private def loadFile(jarPath: String): AppJar = {
     val jarFile = new java.io.File(jarPath)
+    if (!jarFile.exists()) {
+      val error = s"File $jarPath does not exist and cannot submit application"
+      throw new Exception(error)
+    }
     Util.uploadJar(jarFile, jarStoreClient)
   }
 
@@ -157,9 +166,18 @@
     fullName
   }
 
-  private def getMasterClient: MasterClient = {
-    val timeout = Try(config.getInt(Constants.GEARPUMP_MASTERCLIENT_TIMEOUT)).getOrElse(90)
-    new MasterClient(master, akka.util.Timeout(timeout, TimeUnit.SECONDS))
+  private def submitApplication(submitApplication: SubmitApplication): RunningApplication = {
+    val result = ActorUtil.askActor[SubmitApplicationResult](master,
+      submitApplication, masterClientTimeout)
+    val application = result.appId match {
+      case Success(appId) =>
+        // scalastyle:off println
+        Console.println(s"Submit application succeed. The application id is $appId")
+        // scalastyle:on println
+        new RunningApplication(appId, master, masterClientTimeout)
+      case Failure(ex) => throw ex
+    }
+    application
   }
 }
 
diff --git a/core/src/main/scala/org/apache/gearpump/cluster/client/MasterClient.scala b/core/src/main/scala/org/apache/gearpump/cluster/client/MasterClient.scala
deleted file mode 100644
index 77ebedf..0000000
--- a/core/src/main/scala/org/apache/gearpump/cluster/client/MasterClient.scala
+++ /dev/null
@@ -1,81 +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 org.apache.gearpump.cluster.client
-
-import scala.concurrent.duration.Duration
-import scala.concurrent.{Await, Future}
-import scala.util.{Failure, Success}
-
-import akka.actor.ActorRef
-import akka.pattern.ask
-import akka.util.Timeout
-
-import org.apache.gearpump.cluster.ClientToMaster._
-import org.apache.gearpump.cluster.MasterToAppMaster.{AppMastersData, AppMastersDataRequest}
-import org.apache.gearpump.cluster.MasterToClient.{ResolveAppIdResult, ShutdownApplicationResult, SubmitApplicationResult}
-import org.apache.gearpump.cluster.{AppDescription, AppJar}
-
-/**
- * Client to inter-operate with Master node.
- *
- * NOTE: Stateless, thread safe
- */
-class MasterClient(master: ActorRef, timeout: Timeout) {
-  implicit val masterClientTimeout = timeout
-
-  def submitApplication(app: AppDescription, appJar: Option[AppJar]): Int = {
-    val result = Await.result(
-      (master ? SubmitApplication(app, appJar)).asInstanceOf[Future[SubmitApplicationResult]],
-      Duration.Inf)
-    val appId = result.appId match {
-      case Success(appId) =>
-        // scalastyle:off println
-        Console.println(s"Submit application succeed. The application id is $appId")
-        // scalastyle:on println
-        appId
-      case Failure(ex) => throw ex
-    }
-    appId
-  }
-
-  def resolveAppId(appId: Int): ActorRef = {
-    val result = Await.result(
-      (master ? ResolveAppId(appId)).asInstanceOf[Future[ResolveAppIdResult]], Duration.Inf)
-    result.appMaster match {
-      case Success(appMaster) => appMaster
-      case Failure(ex) => throw ex
-    }
-  }
-
-  def shutdownApplication(appId: Int): Unit = {
-    val result = Await.result(
-      (master ? ShutdownApplication(appId)).asInstanceOf[Future[ShutdownApplicationResult]],
-      Duration.Inf)
-    result.appId match {
-      case Success(_) =>
-      case Failure(ex) => throw ex
-    }
-  }
-
-  def listApplications: AppMastersData = {
-    val result = Await.result(
-      (master ? AppMastersDataRequest).asInstanceOf[Future[AppMastersData]], Duration.Inf)
-    result
-  }
-}
\ No newline at end of file
diff --git a/core/src/main/scala/org/apache/gearpump/cluster/client/RunningApplication.scala b/core/src/main/scala/org/apache/gearpump/cluster/client/RunningApplication.scala
new file mode 100644
index 0000000..153c824
--- /dev/null
+++ b/core/src/main/scala/org/apache/gearpump/cluster/client/RunningApplication.scala
@@ -0,0 +1,52 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES 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.gearpump.cluster.client
+
+import akka.pattern.ask
+import akka.actor.ActorRef
+import akka.util.Timeout
+import org.apache.gearpump.cluster.ClientToMaster.{ResolveAppId, ShutdownApplication}
+import org.apache.gearpump.cluster.MasterToClient.{ResolveAppIdResult, ShutdownApplicationResult}
+import org.apache.gearpump.util.ActorUtil
+
+import scala.concurrent.Future
+import scala.util.{Failure, Success}
+import scala.concurrent.ExecutionContext.Implicits.global
+
+class RunningApplication(val appId: Int, master: ActorRef, timeout: Timeout) {
+  lazy val appMaster: Future[ActorRef] = resolveAppMaster(appId)
+
+  def shutDown(): Unit = {
+    val result = ActorUtil.askActor[ShutdownApplicationResult](master,
+      ShutdownApplication(appId), timeout)
+    result.appId match {
+      case Success(_) =>
+      case Failure(ex) => throw ex
+    }
+  }
+
+  def askAppMaster[T](msg: Any): Future[T] = {
+    appMaster.flatMap(_.ask(msg)(timeout).asInstanceOf[Future[T]])
+  }
+
+  private def resolveAppMaster(appId: Int): Future[ActorRef] = {
+    master.ask(ResolveAppId(appId))(timeout).
+      asInstanceOf[Future[ResolveAppIdResult]].map(_.appMaster.get)
+  }
+}
+
diff --git a/daemon/src/main/scala/org/apache/gearpump/cluster/embedded/EmbeddedCluster.scala b/core/src/main/scala/org/apache/gearpump/cluster/embedded/EmbeddedCluster.scala
similarity index 100%
rename from daemon/src/main/scala/org/apache/gearpump/cluster/embedded/EmbeddedCluster.scala
rename to core/src/main/scala/org/apache/gearpump/cluster/embedded/EmbeddedCluster.scala
diff --git a/daemon/src/main/scala/org/apache/gearpump/cluster/main/Local.scala b/core/src/main/scala/org/apache/gearpump/cluster/main/Local.scala
similarity index 99%
rename from daemon/src/main/scala/org/apache/gearpump/cluster/main/Local.scala
rename to core/src/main/scala/org/apache/gearpump/cluster/main/Local.scala
index db71b7b..db2cd8a 100644
--- a/daemon/src/main/scala/org/apache/gearpump/cluster/main/Local.scala
+++ b/core/src/main/scala/org/apache/gearpump/cluster/main/Local.scala
@@ -18,20 +18,19 @@
 
 package org.apache.gearpump.cluster.main
 
-import scala.collection.JavaConverters._
-import scala.concurrent.Await
-import scala.concurrent.duration.Duration
-
 import akka.actor.{ActorSystem, Props}
 import com.typesafe.config.ConfigValueFactory
-import org.slf4j.Logger
-
 import org.apache.gearpump.cluster.ClusterConfig
 import org.apache.gearpump.cluster.master.{Master => MasterActor}
 import org.apache.gearpump.cluster.worker.{Worker => WorkerActor}
 import org.apache.gearpump.util.Constants._
 import org.apache.gearpump.util.LogUtil.ProcessType
 import org.apache.gearpump.util.{ActorUtil, AkkaApp, Constants, LogUtil, Util}
+import org.slf4j.Logger
+
+import scala.collection.JavaConverters._
+import scala.concurrent.Await
+import scala.concurrent.duration.Duration
 
 object Local extends AkkaApp with ArgumentsParser {
   override def akkaConfig: Config = ClusterConfig.master()
diff --git a/daemon/src/main/scala/org/apache/gearpump/cluster/main/Master.scala b/core/src/main/scala/org/apache/gearpump/cluster/main/Master.scala
similarity index 97%
rename from daemon/src/main/scala/org/apache/gearpump/cluster/main/Master.scala
rename to core/src/main/scala/org/apache/gearpump/cluster/main/Master.scala
index f1b9bdf..f758720 100644
--- a/daemon/src/main/scala/org/apache/gearpump/cluster/main/Master.scala
+++ b/core/src/main/scala/org/apache/gearpump/cluster/main/Master.scala
@@ -19,26 +19,26 @@
 package org.apache.gearpump.cluster.main
 
 import java.util.concurrent.TimeUnit
+
+import akka.actor._
+import akka.cluster.ClusterEvent._
+import akka.cluster.{MemberStatus, Member, Cluster}
+import akka.cluster.ddata.DistributedData
+import akka.cluster.singleton.{ClusterSingletonProxySettings, ClusterSingletonProxy, ClusterSingletonManagerSettings, ClusterSingletonManager}
+import com.typesafe.config.ConfigValueFactory
+import org.apache.gearpump.cluster.ClusterConfig
+import org.apache.gearpump.cluster.master.Master.MasterListUpdated
+import org.apache.gearpump.cluster.master.{Master => MasterActor, MasterNode}
+import org.apache.gearpump.util.Constants._
+import org.apache.gearpump.util.LogUtil.ProcessType
+import org.apache.gearpump.util.{AkkaApp, Constants, LogUtil}
+import org.slf4j.Logger
+
 import scala.collection.JavaConverters._
 import scala.collection.immutable
 import scala.concurrent.Await
 import scala.concurrent.duration._
 
-import akka.actor._
-import akka.cluster.ClusterEvent._
-import akka.cluster.ddata.DistributedData
-import akka.cluster.singleton.{ClusterSingletonManager, ClusterSingletonManagerSettings, ClusterSingletonProxy, ClusterSingletonProxySettings}
-import akka.cluster.{Cluster, Member, MemberStatus}
-import com.typesafe.config.ConfigValueFactory
-import org.slf4j.Logger
-
-import org.apache.gearpump.cluster.ClusterConfig
-import org.apache.gearpump.cluster.master.{Master => MasterActor, MasterNode}
-import org.apache.gearpump.cluster.master.Master.MasterListUpdated
-import org.apache.gearpump.util.Constants._
-import org.apache.gearpump.util.LogUtil.ProcessType
-import org.apache.gearpump.util.{AkkaApp, Constants, LogUtil}
-
 object Master extends AkkaApp with ArgumentsParser {
 
   private var LOG: Logger = LogUtil.getLogger(getClass)
diff --git a/daemon/src/main/scala/org/apache/gearpump/cluster/main/Worker.scala b/core/src/main/scala/org/apache/gearpump/cluster/main/Worker.scala
similarity index 99%
rename from daemon/src/main/scala/org/apache/gearpump/cluster/main/Worker.scala
rename to core/src/main/scala/org/apache/gearpump/cluster/main/Worker.scala
index 58a9dec..3d8d823 100644
--- a/daemon/src/main/scala/org/apache/gearpump/cluster/main/Worker.scala
+++ b/core/src/main/scala/org/apache/gearpump/cluster/main/Worker.scala
@@ -18,21 +18,20 @@
 
 package org.apache.gearpump.cluster.main
 
+import akka.actor.{ActorSystem, Props}
+import org.apache.gearpump.cluster.ClusterConfig
+import org.apache.gearpump.cluster.master.MasterProxy
+import org.apache.gearpump.cluster.worker.{Worker => WorkerActor}
+import org.apache.gearpump.transport.HostPort
+import org.apache.gearpump.util.Constants._
+import org.apache.gearpump.util.LogUtil.ProcessType
+import org.apache.gearpump.util.{AkkaApp, LogUtil}
+import org.slf4j.Logger
+
 import scala.collection.JavaConverters._
 import scala.concurrent.Await
 import scala.concurrent.duration.Duration
 
-import akka.actor.{ActorSystem, Props}
-import org.slf4j.Logger
-
-import org.apache.gearpump.cluster.ClusterConfig
-import org.apache.gearpump.cluster.master.MasterProxy
-import org.apache.gearpump.transport.HostPort
-import org.apache.gearpump.util.Constants._
-import org.apache.gearpump.cluster.worker.{Worker => WorkerActor}
-import org.apache.gearpump.util.LogUtil.ProcessType
-import org.apache.gearpump.util.{AkkaApp, LogUtil}
-
 /** Tool to start a worker daemon process */
 object Worker extends AkkaApp with ArgumentsParser {
   protected override def akkaConfig = ClusterConfig.worker()
diff --git a/daemon/src/main/scala/org/apache/gearpump/cluster/master/AppManager.scala b/core/src/main/scala/org/apache/gearpump/cluster/master/AppManager.scala
similarity index 99%
rename from daemon/src/main/scala/org/apache/gearpump/cluster/master/AppManager.scala
rename to core/src/main/scala/org/apache/gearpump/cluster/master/AppManager.scala
index 9a3a119..0ae7365 100644
--- a/daemon/src/main/scala/org/apache/gearpump/cluster/master/AppManager.scala
+++ b/core/src/main/scala/org/apache/gearpump/cluster/master/AppManager.scala
@@ -18,14 +18,8 @@
 
 package org.apache.gearpump.cluster.master
 
-import scala.concurrent.Future
-import scala.concurrent.duration._
-import scala.util.{Failure, Success}
-
 import akka.actor._
 import akka.pattern.ask
-import org.slf4j.Logger
-
 import org.apache.gearpump.cluster.AppMasterToMaster.{AppDataSaved, SaveAppDataFailed, _}
 import org.apache.gearpump.cluster.AppMasterToWorker._
 import org.apache.gearpump.cluster.ClientToMaster._
@@ -38,6 +32,11 @@
 import org.apache.gearpump.cluster.master.Master._
 import org.apache.gearpump.util.Constants._
 import org.apache.gearpump.util.{ActorUtil, TimeOutScheduler, Util, _}
+import org.slf4j.Logger
+
+import scala.concurrent.Future
+import scala.concurrent.duration._
+import scala.util.{Failure, Success}
 
 /**
  * AppManager is dedicated child of Master to manager all applications.
diff --git a/daemon/src/main/scala/org/apache/gearpump/cluster/master/InMemoryKVService.scala b/core/src/main/scala/org/apache/gearpump/cluster/master/InMemoryKVService.scala
similarity index 98%
rename from daemon/src/main/scala/org/apache/gearpump/cluster/master/InMemoryKVService.scala
rename to core/src/main/scala/org/apache/gearpump/cluster/master/InMemoryKVService.scala
index 3e54214..fd19bad 100644
--- a/daemon/src/main/scala/org/apache/gearpump/cluster/master/InMemoryKVService.scala
+++ b/core/src/main/scala/org/apache/gearpump/cluster/master/InMemoryKVService.scala
@@ -19,16 +19,16 @@
 package org.apache.gearpump.cluster.master
 
 import java.util.concurrent.TimeUnit
-import scala.concurrent.TimeoutException
-import scala.concurrent.duration.Duration
 
 import akka.actor._
 import akka.cluster.Cluster
+import akka.cluster.ddata.{LWWMap, LWWMapKey, DistributedData}
 import akka.cluster.ddata.Replicator._
-import akka.cluster.ddata.{DistributedData, LWWMap, LWWMapKey}
+import org.apache.gearpump.util.LogUtil
 import org.slf4j.Logger
 
-import org.apache.gearpump.util.LogUtil
+import scala.concurrent.TimeoutException
+import scala.concurrent.duration.Duration
 
 /**
  * A replicated simple in-memory KV service. The replications are stored on all masters.
diff --git a/daemon/src/main/scala/org/apache/gearpump/cluster/master/Master.scala b/core/src/main/scala/org/apache/gearpump/cluster/master/Master.scala
similarity index 100%
rename from daemon/src/main/scala/org/apache/gearpump/cluster/master/Master.scala
rename to core/src/main/scala/org/apache/gearpump/cluster/master/Master.scala
diff --git a/daemon/src/main/scala/org/apache/gearpump/cluster/scheduler/PriorityScheduler.scala b/core/src/main/scala/org/apache/gearpump/cluster/scheduler/PriorityScheduler.scala
similarity index 99%
rename from daemon/src/main/scala/org/apache/gearpump/cluster/scheduler/PriorityScheduler.scala
rename to core/src/main/scala/org/apache/gearpump/cluster/scheduler/PriorityScheduler.scala
index 1429694..623e3ff 100644
--- a/daemon/src/main/scala/org/apache/gearpump/cluster/scheduler/PriorityScheduler.scala
+++ b/core/src/main/scala/org/apache/gearpump/cluster/scheduler/PriorityScheduler.scala
@@ -18,16 +18,14 @@
 
 package org.apache.gearpump.cluster.scheduler
 
-import org.apache.gearpump.cluster.worker.WorkerId
-
-import scala.collection.mutable
-
 import akka.actor.ActorRef
-
 import org.apache.gearpump.cluster.AppMasterToMaster.RequestResource
 import org.apache.gearpump.cluster.MasterToAppMaster.ResourceAllocated
 import org.apache.gearpump.cluster.scheduler.Relaxation._
 import org.apache.gearpump.cluster.scheduler.Scheduler.PendingRequest
+import org.apache.gearpump.cluster.worker.WorkerId
+
+import scala.collection.mutable
 
 /** Assign resource to application based on the priority of the application */
 class PriorityScheduler extends Scheduler {
diff --git a/daemon/src/main/scala/org/apache/gearpump/cluster/scheduler/Scheduler.scala b/core/src/main/scala/org/apache/gearpump/cluster/scheduler/Scheduler.scala
similarity index 99%
rename from daemon/src/main/scala/org/apache/gearpump/cluster/scheduler/Scheduler.scala
rename to core/src/main/scala/org/apache/gearpump/cluster/scheduler/Scheduler.scala
index 7187c1a..ec9f1ba 100644
--- a/daemon/src/main/scala/org/apache/gearpump/cluster/scheduler/Scheduler.scala
+++ b/core/src/main/scala/org/apache/gearpump/cluster/scheduler/Scheduler.scala
@@ -17,19 +17,17 @@
  */
 package org.apache.gearpump.cluster.scheduler
 
-import org.apache.gearpump.cluster.worker.WorkerId
-
-import scala.collection.mutable
-
 import akka.actor.{Actor, ActorRef}
-import org.slf4j.Logger
-
 import org.apache.gearpump.TimeStamp
 import org.apache.gearpump.cluster.MasterToWorker.{UpdateResourceFailed, UpdateResourceSucceed, WorkerRegistered}
 import org.apache.gearpump.cluster.WorkerToMaster.ResourceUpdate
 import org.apache.gearpump.cluster.master.Master.WorkerTerminated
 import org.apache.gearpump.cluster.scheduler.Scheduler.ApplicationFinished
+import org.apache.gearpump.cluster.worker.WorkerId
 import org.apache.gearpump.util.LogUtil
+import org.slf4j.Logger
+
+import scala.collection.mutable
 
 /**
  * Scheduler schedule resource for different applications.
diff --git a/daemon/src/main/scala/org/apache/gearpump/cluster/worker/DefaultExecutorProcessLauncher.scala b/core/src/main/scala/org/apache/gearpump/cluster/worker/DefaultExecutorProcessLauncher.scala
similarity index 99%
rename from daemon/src/main/scala/org/apache/gearpump/cluster/worker/DefaultExecutorProcessLauncher.scala
rename to core/src/main/scala/org/apache/gearpump/cluster/worker/DefaultExecutorProcessLauncher.scala
index b4e6f9e..3d5b0af 100644
--- a/daemon/src/main/scala/org/apache/gearpump/cluster/worker/DefaultExecutorProcessLauncher.scala
+++ b/core/src/main/scala/org/apache/gearpump/cluster/worker/DefaultExecutorProcessLauncher.scala
@@ -20,10 +20,9 @@
 import java.io.File
 
 import com.typesafe.config.Config
-import org.slf4j.Logger
-
 import org.apache.gearpump.cluster.scheduler.Resource
 import org.apache.gearpump.util.{LogUtil, RichProcess, Util}
+import org.slf4j.Logger
 
 /** Launcher to start an executor process */
 class DefaultExecutorProcessLauncher(val config: Config) extends ExecutorProcessLauncher {
diff --git a/daemon/src/main/scala/org/apache/gearpump/cluster/worker/Worker.scala b/core/src/main/scala/org/apache/gearpump/cluster/worker/Worker.scala
similarity index 97%
rename from daemon/src/main/scala/org/apache/gearpump/cluster/worker/Worker.scala
rename to core/src/main/scala/org/apache/gearpump/cluster/worker/Worker.scala
index 1b52e5d..447b034 100644
--- a/daemon/src/main/scala/org/apache/gearpump/cluster/worker/Worker.scala
+++ b/core/src/main/scala/org/apache/gearpump/cluster/worker/Worker.scala
@@ -22,34 +22,33 @@
 import java.lang.management.ManagementFactory
 import java.net.URL
 import java.util.concurrent.{Executors, TimeUnit}
-import org.apache.gearpump.cluster.worker.Worker.ExecutorWatcher
-
-import scala.concurrent.duration._
-import scala.concurrent.{ExecutionContext, Future, Promise}
-import scala.util.{Failure, Success, Try}
 
 import akka.actor.SupervisorStrategy.Stop
 import akka.actor._
-import com.typesafe.config.{ConfigValueFactory, Config, ConfigFactory}
-import org.slf4j.Logger
-
+import com.typesafe.config.{Config, ConfigFactory, ConfigValueFactory}
 import org.apache.gearpump.cluster.AppMasterToMaster.{GetWorkerData, WorkerData}
 import org.apache.gearpump.cluster.AppMasterToWorker._
 import org.apache.gearpump.cluster.ClientToMaster.{QueryHistoryMetrics, QueryWorkerConfig}
 import org.apache.gearpump.cluster.MasterToClient.{HistoryMetrics, HistoryMetricsItem, WorkerConfig}
-import org.apache.gearpump.cluster.MasterToWorker._
+import org.apache.gearpump.cluster.MasterToWorker.{UpdateResourceSucceed, UpdateResourceFailed, WorkerRegistered}
 import org.apache.gearpump.cluster.WorkerToAppMaster._
-import org.apache.gearpump.cluster.WorkerToMaster._
+import org.apache.gearpump.cluster.WorkerToMaster.{RegisterNewWorker, RegisterWorker, ResourceUpdate}
 import org.apache.gearpump.cluster.master.Master.MasterInfo
 import org.apache.gearpump.cluster.scheduler.Resource
+import org.apache.gearpump.cluster.worker.Worker.ExecutorWatcher
 import org.apache.gearpump.cluster.{ClusterConfig, ExecutorJVMConfig}
-import org.apache.gearpump.jarstore.{JarStoreClient, JarStoreServer}
+import org.apache.gearpump.jarstore.JarStoreClient
 import org.apache.gearpump.metrics.Metrics.ReportMetrics
 import org.apache.gearpump.metrics.{JvmMetricsSet, Metrics, MetricsReporterService}
 import org.apache.gearpump.util.ActorSystemBooter.Daemon
 import org.apache.gearpump.util.Constants._
 import org.apache.gearpump.util.HistoryMetricsService.HistoryMetricsConfig
 import org.apache.gearpump.util.{TimeOutScheduler, _}
+import org.slf4j.Logger
+
+import scala.concurrent.duration._
+import scala.concurrent.{ExecutionContext, Future, Promise}
+import scala.util.{Failure, Success, Try}
 
 /**
  * Worker is used to track the resource on single machine, it is like
@@ -519,8 +518,7 @@
     }
 
     // The folders are under ${GEARPUMP_HOME}
-    val daemonPathPattern = List("lib" + File.separator + "daemon", "lib" +
-      File.separator + "yarn")
+    val daemonPathPattern = List("lib" + File.separator + "yarn")
 
     override def receive: Receive = {
       case ShutdownExecutor(appId, executorId, reason: String) =>
diff --git a/core/src/main/scala/org/apache/gearpump/metrics/AkkaReporter.scala b/core/src/main/scala/org/apache/gearpump/metrics/AkkaReporter.scala
index acd9493..9092beb 100644
--- a/core/src/main/scala/org/apache/gearpump/metrics/AkkaReporter.scala
+++ b/core/src/main/scala/org/apache/gearpump/metrics/AkkaReporter.scala
@@ -22,7 +22,7 @@
 
 import akka.actor.{ActorRef, ActorSystem}
 
-import org.apache.gearpump.codahale.metrics.{Gauge => CodaGauge, MetricRegistry}
+import com.codahale.metrics.{Gauge => CodaGauge, MetricRegistry}
 import org.apache.gearpump.metrics.Metrics.{Counter => CounterData, Gauge => GaugeData, Histogram => HistogramData, Meter => MeterData}
 import org.apache.gearpump.metrics.MetricsReporterService.ReportTo
 import org.apache.gearpump.util.LogUtil
diff --git a/core/src/main/scala/org/apache/gearpump/metrics/Counter.scala b/core/src/main/scala/org/apache/gearpump/metrics/Counter.scala
index 2a4a195..e778e3d 100644
--- a/core/src/main/scala/org/apache/gearpump/metrics/Counter.scala
+++ b/core/src/main/scala/org/apache/gearpump/metrics/Counter.scala
@@ -18,10 +18,10 @@
 
 package org.apache.gearpump.metrics
 
-import org.apache.gearpump.codahale.metrics.{Counter => CodaHaleCounter}
+import com.codahale.metrics.{Counter => CodaHaleCounter}
 
 /**
- * @see org.apache.gearpump.codahale.metrics.Counter
+ * @see com.codahale.metrics.Counter
  */
 class Counter(val name: String, counter: CodaHaleCounter, sampleRate: Int = 1) {
   private var sampleCount = 0L
diff --git a/core/src/main/scala/org/apache/gearpump/metrics/Histogram.scala b/core/src/main/scala/org/apache/gearpump/metrics/Histogram.scala
index 9d1e500..3483c1e 100644
--- a/core/src/main/scala/org/apache/gearpump/metrics/Histogram.scala
+++ b/core/src/main/scala/org/apache/gearpump/metrics/Histogram.scala
@@ -18,10 +18,10 @@
 
 package org.apache.gearpump.metrics
 
-import org.apache.gearpump.codahale.metrics.{Histogram => CodaHaleHistogram}
+import com.codahale.metrics.{Histogram => CodaHaleHistogram}
 
 /**
- * @see org.apache.gearpump.codahale.metrics.Histogram
+ * @see com.codahale.metrics.Histogram
  */
 class Histogram(val name: String, histogram: CodaHaleHistogram, sampleRate: Int = 1) {
   private var sampleCount = 0L
diff --git a/core/src/main/scala/org/apache/gearpump/metrics/JvmMetricsSet.scala b/core/src/main/scala/org/apache/gearpump/metrics/JvmMetricsSet.scala
index 436d14f..56b4743 100644
--- a/core/src/main/scala/org/apache/gearpump/metrics/JvmMetricsSet.scala
+++ b/core/src/main/scala/org/apache/gearpump/metrics/JvmMetricsSet.scala
@@ -21,8 +21,8 @@
 import java.util
 import scala.collection.JavaConverters._
 
-import org.apache.gearpump.codahale.metrics.jvm.{MemoryUsageGaugeSet, ThreadStatesGaugeSet}
-import org.apache.gearpump.codahale.metrics.{Metric, MetricSet}
+import com.codahale.metrics.jvm.{MemoryUsageGaugeSet, ThreadStatesGaugeSet}
+import com.codahale.metrics.{Metric, MetricSet}
 
 class JvmMetricsSet(name: String) extends MetricSet {
 
diff --git a/core/src/main/scala/org/apache/gearpump/metrics/Meter.scala b/core/src/main/scala/org/apache/gearpump/metrics/Meter.scala
index d4b7871..55aa73f 100644
--- a/core/src/main/scala/org/apache/gearpump/metrics/Meter.scala
+++ b/core/src/main/scala/org/apache/gearpump/metrics/Meter.scala
@@ -18,7 +18,7 @@
 
 package org.apache.gearpump.metrics
 
-import org.apache.gearpump.codahale.metrics.{Meter => CodaHaleMeter}
+import com.codahale.metrics.{Meter => CodaHaleMeter}
 
 /** See org.apache.gearpump.codahale.metrics.Meter */
 class Meter(val name: String, meter: CodaHaleMeter, sampleRate: Int = 1) {
diff --git a/core/src/main/scala/org/apache/gearpump/metrics/Metrics.scala b/core/src/main/scala/org/apache/gearpump/metrics/Metrics.scala
index 1ee3798..3737361 100644
--- a/core/src/main/scala/org/apache/gearpump/metrics/Metrics.scala
+++ b/core/src/main/scala/org/apache/gearpump/metrics/Metrics.scala
@@ -23,7 +23,7 @@
 import akka.actor._
 import org.slf4j.Logger
 
-import org.apache.gearpump.codahale.metrics._
+import com.codahale.metrics._
 import org.apache.gearpump.metrics
 import org.apache.gearpump.util.LogUtil
 
diff --git a/core/src/main/scala/org/apache/gearpump/metrics/MetricsReporterService.scala b/core/src/main/scala/org/apache/gearpump/metrics/MetricsReporterService.scala
index 94aa114..620dc61 100644
--- a/core/src/main/scala/org/apache/gearpump/metrics/MetricsReporterService.scala
+++ b/core/src/main/scala/org/apache/gearpump/metrics/MetricsReporterService.scala
@@ -20,12 +20,11 @@
 
 import java.net.InetSocketAddress
 import java.util.concurrent.TimeUnit
+
 import scala.concurrent.duration._
-
 import akka.actor.{Actor, ActorRef}
-
-import org.apache.gearpump.codahale.metrics.graphite.{Graphite, GraphiteReporter}
-import org.apache.gearpump.codahale.metrics.{MetricFilter, Slf4jReporter}
+import com.codahale.metrics.{MetricFilter, Slf4jReporter}
+import com.codahale.metrics.graphite.{Graphite, GraphiteReporter}
 import org.apache.gearpump.metrics.Metrics.{DemandMoreMetrics, ReportMetrics}
 import org.apache.gearpump.metrics.MetricsReporterService.ReportTo
 import org.apache.gearpump.util.Constants._
diff --git a/core/src/main/scala/org/apache/gearpump/serializer/FastKryoSerializer.scala b/core/src/main/scala/org/apache/gearpump/serializer/FastKryoSerializer.scala
index f258c0f..cb3d2fd 100644
--- a/core/src/main/scala/org/apache/gearpump/serializer/FastKryoSerializer.scala
+++ b/core/src/main/scala/org/apache/gearpump/serializer/FastKryoSerializer.scala
@@ -19,19 +19,18 @@
 package org.apache.gearpump.serializer
 
 import akka.actor.ExtendedActorSystem
-
-import org.apache.gearpump.esotericsoftware.kryo.Kryo.DefaultInstantiatorStrategy
-import org.apache.gearpump.objenesis.strategy.StdInstantiatorStrategy
-import org.apache.gearpump.romix.serialization.kryo.KryoSerializerWrapper
+import com.esotericsoftware.kryo.Kryo.DefaultInstantiatorStrategy
+import com.romix.akka.serialization.kryo.{KryoBasedSerializer, KryoSerializer}
 import org.apache.gearpump.serializer.FastKryoSerializer.KryoSerializationException
 import org.apache.gearpump.util.LogUtil
+import org.objenesis.strategy.StdInstantiatorStrategy
 
 class FastKryoSerializer(system: ExtendedActorSystem) extends Serializer {
 
   private val LOG = LogUtil.getLogger(getClass)
   private val config = system.settings.config
 
-  private val kryoSerializer = new KryoSerializerWrapper(system)
+  private val kryoSerializer: KryoBasedSerializer = new KryoSerializer(system).serializer
   private val kryo = kryoSerializer.kryo
   val strategy = new DefaultInstantiatorStrategy
   strategy.setFallbackInstantiatorStrategy(new StdInstantiatorStrategy)
@@ -40,7 +39,7 @@
 
   override def serialize(message: Any): Array[Byte] = {
     try {
-      kryoSerializer.toBinary(message)
+      kryoSerializer.toBinary(message.asInstanceOf[AnyRef])
     } catch {
       case ex: java.lang.IllegalArgumentException =>
         val clazz = message.getClass
diff --git a/core/src/main/scala/org/apache/gearpump/serializer/GearpumpSerialization.scala b/core/src/main/scala/org/apache/gearpump/serializer/GearpumpSerialization.scala
index 524089d..45a5481 100644
--- a/core/src/main/scala/org/apache/gearpump/serializer/GearpumpSerialization.scala
+++ b/core/src/main/scala/org/apache/gearpump/serializer/GearpumpSerialization.scala
@@ -18,10 +18,9 @@
 
 package org.apache.gearpump.serializer
 
+import com.esotericsoftware.kryo.{Kryo, Serializer => KryoSerializer}
 import com.typesafe.config.Config
 import org.slf4j.Logger
-
-import org.apache.gearpump.esotericsoftware.kryo.{Kryo, Serializer => KryoSerializer}
 import org.apache.gearpump.util.{Constants, LogUtil}
 
 class GearpumpSerialization(config: Config) {
diff --git a/core/src/main/scala/org/apache/gearpump/util/ActorUtil.scala b/core/src/main/scala/org/apache/gearpump/util/ActorUtil.scala
index 09f2969..82c7fe2 100644
--- a/core/src/main/scala/org/apache/gearpump/util/ActorUtil.scala
+++ b/core/src/main/scala/org/apache/gearpump/util/ActorUtil.scala
@@ -21,13 +21,12 @@
 import org.apache.gearpump.cluster.AppMasterContext
 import org.apache.gearpump.cluster.worker.WorkerId
 
-import scala.concurrent.{ExecutionContext, Future}
-
+import scala.concurrent.{Await, ExecutionContext, Future}
 import akka.actor.Actor.Receive
 import akka.actor._
 import akka.pattern.ask
 import org.slf4j.Logger
-
+import akka.util.Timeout
 import org.apache.gearpump.cluster.AppMasterToMaster.{ActivateAppMaster, GetAllWorkers}
 import org.apache.gearpump.cluster.ClientToMaster.{ResolveAppId, ResolveWorkerId}
 import org.apache.gearpump.cluster.MasterToAppMaster.WorkerList
@@ -36,6 +35,8 @@
 import org.apache.gearpump.cluster.scheduler.{Relaxation, Resource, ResourceRequest}
 import org.apache.gearpump.transport.HostPort
 
+import scala.concurrent.duration.Duration
+
 object ActorUtil {
   private val LOG: Logger = LogUtil.getLogger(getClass)
 
@@ -136,4 +137,13 @@
     implicit val timeout = Constants.FUTURE_TIMEOUT
     (actor ? msg).asInstanceOf[Future[T]]
   }
+
+  def askActor[T](actor: ActorRef, msg: Any, timeout: Timeout)(implicit ex: ExecutionContext): T = {
+    askActor(actor, msg, timeout, ActorRef.noSender)
+  }
+
+  def askActor[T](actor: ActorRef, msg: Any, timeout: Timeout, sender: ActorRef)
+    (implicit ex: ExecutionContext): T = {
+    Await.result(actor.ask(msg)(timeout, sender).asInstanceOf[Future[T]], Duration.Inf)
+  }
 }
diff --git a/core/src/main/scala/org/apache/gearpump/util/Constants.scala b/core/src/main/scala/org/apache/gearpump/util/Constants.scala
index dba5a1f..c98726e 100644
--- a/core/src/main/scala/org/apache/gearpump/util/Constants.scala
+++ b/core/src/main/scala/org/apache/gearpump/util/Constants.scala
@@ -20,8 +20,6 @@
 
 import java.util.concurrent.TimeUnit
 
-import org.apache.gearpump.partitioner._
-
 object Constants {
   val MASTER_WATCHER = "masterwatcher"
   val SINGLETON_MANAGER = "singleton"
@@ -140,14 +138,6 @@
   val GEARPUMP_SERVICE_SUPERVISOR_PATH = "gearpump.services.supervisor-actor-path"
   val GEARPUMP_SERVICE_RENDER_CONFIG_CONCISE = "gearpump.services.config-render-option-concise"
 
-  // The partitioners provided by Gearpump
-  val BUILTIN_PARTITIONERS = Array(
-    classOf[BroadcastPartitioner],
-    classOf[CoLocationPartitioner],
-    classOf[HashPartitioner],
-    classOf[ShuffleGroupingPartitioner],
-    classOf[ShufflePartitioner])
-
   // Security related
   val GEARPUMP_KEYTAB_FILE = "gearpump.keytab.file"
   val GEARPUMP_KERBEROS_PRINCIPAL = "gearpump.kerberos.principal"
diff --git a/core/src/main/scala/org/apache/gearpump/util/FileUtils.scala b/core/src/main/scala/org/apache/gearpump/util/FileUtils.scala
index e3df37b..283a64a 100644
--- a/core/src/main/scala/org/apache/gearpump/util/FileUtils.scala
+++ b/core/src/main/scala/org/apache/gearpump/util/FileUtils.scala
@@ -21,7 +21,7 @@
 import java.io.{File, IOException}
 import java.nio.charset.Charset
 
-import org.apache.gearpump.google.common.io.Files
+import com.google.common.io.Files
 
 object FileUtils {
   private val UTF8 = Charset.forName("UTF-8")
diff --git a/core/src/main/scala/org/apache/gearpump/util/Util.scala b/core/src/main/scala/org/apache/gearpump/util/Util.scala
index 0faa46a..8ee0e26 100644
--- a/core/src/main/scala/org/apache/gearpump/util/Util.scala
+++ b/core/src/main/scala/org/apache/gearpump/util/Util.scala
@@ -66,8 +66,14 @@
   def startProcess(options: Array[String], classPath: Array[String], mainClass: String,
       arguments: Array[String]): RichProcess = {
     val java = System.getProperty("java.home") + "/bin/java"
-    val command = List(java) ++ options ++
-      List("-cp", classPath.mkString(File.pathSeparator), mainClass) ++ arguments
+
+    val command = List(java) ++
+      // java.lang.VerifyError will be caused without "-noverify"
+      // TODO: investigate the cause and remove this
+      Array("-noverify") ++
+      options ++
+      List("-cp", classPath.mkString(File.pathSeparator), mainClass) ++
+      arguments
     LOG.info(s"Starting executor process java $mainClass ${arguments.mkString(" ")} " +
       s"\n ${options.mkString(" ")}")
     val logger = new ProcessLogRedirector()
diff --git a/daemon/src/main/resources/META-INF/services/org.apache.gearpump.jarstore.JarStore b/core/src/test/resources/META-INF/services/org.apache.gearpump.jarstore.JarStore
similarity index 94%
copy from daemon/src/main/resources/META-INF/services/org.apache.gearpump.jarstore.JarStore
copy to core/src/test/resources/META-INF/services/org.apache.gearpump.jarstore.JarStore
index e173a8a..c64d444 100644
--- a/daemon/src/main/resources/META-INF/services/org.apache.gearpump.jarstore.JarStore
+++ b/core/src/test/resources/META-INF/services/org.apache.gearpump.jarstore.JarStore
@@ -16,5 +16,4 @@
 # limitations under the License.
 #
 
-org.apache.gearpump.jarstore.local.LocalJarStore
-org.apache.gearpump.jarstore.dfs.DFSJarStore
\ No newline at end of file
+org.apache.gearpump.jarstore.local.LocalJarStore
\ No newline at end of file
diff --git a/daemon/src/test/scala/org/apache/gearpump/cluster/MiniCluster.scala b/core/src/test/scala/org/apache/gearpump/cluster/MiniCluster.scala
similarity index 99%
rename from daemon/src/test/scala/org/apache/gearpump/cluster/MiniCluster.scala
rename to core/src/test/scala/org/apache/gearpump/cluster/MiniCluster.scala
index a6b75cb..0a22245 100644
--- a/daemon/src/test/scala/org/apache/gearpump/cluster/MiniCluster.scala
+++ b/core/src/test/scala/org/apache/gearpump/cluster/MiniCluster.scala
@@ -17,20 +17,19 @@
  */
 package org.apache.gearpump.cluster
 
-import scala.concurrent.duration.Duration
-import scala.concurrent.{Await, Future}
-
 import akka.actor.{Actor, ActorRef, ActorSystem, Props}
 import akka.pattern.ask
 import akka.testkit.TestActorRef
 import com.typesafe.config.ConfigValueFactory
-
 import org.apache.gearpump.cluster.AppMasterToMaster.GetAllWorkers
 import org.apache.gearpump.cluster.MasterToAppMaster.WorkerList
 import org.apache.gearpump.cluster.master.Master
 import org.apache.gearpump.cluster.worker.Worker
 import org.apache.gearpump.util.Constants
 
+import scala.concurrent.duration.Duration
+import scala.concurrent.{Await, Future}
+
 class MiniCluster {
   private val mockMasterIP = "127.0.0.1"
 
diff --git a/daemon/src/test/scala/org/apache/gearpump/cluster/master/AppManagerSpec.scala b/core/src/test/scala/org/apache/gearpump/cluster/appmaster/AppManagerSpec.scala
similarity index 97%
rename from daemon/src/test/scala/org/apache/gearpump/cluster/master/AppManagerSpec.scala
rename to core/src/test/scala/org/apache/gearpump/cluster/appmaster/AppManagerSpec.scala
index 58e3593..f9b0762 100644
--- a/daemon/src/test/scala/org/apache/gearpump/cluster/master/AppManagerSpec.scala
+++ b/core/src/test/scala/org/apache/gearpump/cluster/appmaster/AppManagerSpec.scala
@@ -16,24 +16,23 @@
  * limitations under the License.
  */
 
-package org.apache.gearpump.cluster.master
-
-import scala.util.Success
+package org.apache.gearpump.cluster.appmaster
 
 import akka.actor.{Actor, ActorRef, Props}
 import akka.testkit.TestProbe
 import com.typesafe.config.Config
-import org.scalatest.{BeforeAndAfterEach, FlatSpec, Matchers}
-
 import org.apache.gearpump.cluster.AppMasterToMaster.{AppDataSaved, _}
 import org.apache.gearpump.cluster.ClientToMaster.{ResolveAppId, ShutdownApplication, SubmitApplication}
 import org.apache.gearpump.cluster.MasterToAppMaster.{AppMasterData, AppMasterRegistered, AppMastersData, AppMastersDataRequest, _}
 import org.apache.gearpump.cluster.MasterToClient.{ResolveAppIdResult, ShutdownApplicationResult, SubmitApplicationResult}
-import org.apache.gearpump.cluster.appmaster.{AppMasterRuntimeInfo, ApplicationState}
+import org.apache.gearpump.cluster.master.{AppMasterLauncherFactory, AppManager}
 import org.apache.gearpump.cluster.master.AppManager._
 import org.apache.gearpump.cluster.master.InMemoryKVService.{GetKV, GetKVSuccess, PutKV, PutKVSuccess}
 import org.apache.gearpump.cluster.{TestUtil, _}
 import org.apache.gearpump.util.LogUtil
+import org.scalatest.{BeforeAndAfterEach, FlatSpec, Matchers}
+
+import scala.util.Success
 
 class AppManagerSpec extends FlatSpec with Matchers with BeforeAndAfterEach with MasterHarness {
   var kvService: TestProbe = null
@@ -166,7 +165,6 @@
 }
 
 class DummyAppMasterLauncherFactory(test: TestProbe) extends AppMasterLauncherFactory {
-
   override def props(appId: Int, executorId: Int, app: AppDescription, jar: Option[AppJar],
       username: String, master: ActorRef, client: Option[ActorRef]): Props = {
     Props(new DummyAppMasterLauncher(test, appId))
@@ -174,8 +172,8 @@
 }
 
 class DummyAppMasterLauncher(test: TestProbe, appId: Int) extends Actor {
-
   test.ref ! LauncherStarted(appId)
+  
   override def receive: Receive = {
     case any: Any => test.ref forward any
   }
diff --git a/daemon/src/test/scala/org/apache/gearpump/cluster/master/InMemoryKVServiceSpec.scala b/core/src/test/scala/org/apache/gearpump/cluster/appmaster/InMemoryKVServiceSpec.scala
similarity index 94%
rename from daemon/src/test/scala/org/apache/gearpump/cluster/master/InMemoryKVServiceSpec.scala
rename to core/src/test/scala/org/apache/gearpump/cluster/appmaster/InMemoryKVServiceSpec.scala
index 325a484..d3e739f 100644
--- a/daemon/src/test/scala/org/apache/gearpump/cluster/master/InMemoryKVServiceSpec.scala
+++ b/core/src/test/scala/org/apache/gearpump/cluster/appmaster/InMemoryKVServiceSpec.scala
@@ -16,17 +16,17 @@
  * limitations under the License.
  */
 
-package org.apache.gearpump.cluster.master
-
-import scala.concurrent.duration._
+package org.apache.gearpump.cluster.appmaster
 
 import akka.actor.Props
 import akka.testkit.TestProbe
 import com.typesafe.config.Config
-import org.scalatest.{BeforeAndAfterEach, FlatSpec, Matchers}
-
+import org.apache.gearpump.cluster.master.InMemoryKVService
 import org.apache.gearpump.cluster.master.InMemoryKVService._
 import org.apache.gearpump.cluster.{MasterHarness, TestUtil}
+import org.scalatest.{BeforeAndAfterEach, FlatSpec, Matchers}
+
+import scala.concurrent.duration._
 
 class InMemoryKVServiceSpec
   extends FlatSpec with Matchers with BeforeAndAfterEach with MasterHarness {
diff --git a/core/src/test/scala/org/apache/gearpump/cluster/client/RunningApplicationSpec.scala b/core/src/test/scala/org/apache/gearpump/cluster/client/RunningApplicationSpec.scala
new file mode 100644
index 0000000..5f0d5e4
--- /dev/null
+++ b/core/src/test/scala/org/apache/gearpump/cluster/client/RunningApplicationSpec.scala
@@ -0,0 +1,99 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES 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.gearpump.cluster.client
+
+import java.util.concurrent.TimeUnit
+
+import akka.actor.ActorSystem
+import akka.testkit.TestProbe
+import akka.util.Timeout
+import org.apache.gearpump.cluster.ClientToMaster.{ResolveAppId, ShutdownApplication}
+import org.apache.gearpump.cluster.MasterToClient.{ResolveAppIdResult, ShutdownApplicationResult}
+import org.apache.gearpump.cluster.TestUtil
+import org.apache.gearpump.cluster.client.RunningApplicationSpec.{MockAskAppMasterRequest, MockAskAppMasterResponse}
+import org.scalatest.{BeforeAndAfterAll, FlatSpec, Matchers}
+
+import scala.concurrent.{Await, Future}
+import scala.concurrent.duration.Duration
+import scala.util.{Failure, Success}
+import scala.concurrent.ExecutionContext.Implicits.global
+
+class RunningApplicationSpec extends FlatSpec with Matchers with BeforeAndAfterAll {
+  implicit var system: ActorSystem = _
+
+  override def beforeAll(): Unit = {
+    system = ActorSystem("test", TestUtil.DEFAULT_CONFIG)
+  }
+
+  override def afterAll(): Unit = {
+    system.terminate()
+    Await.result(system.whenTerminated, Duration.Inf)
+  }
+
+  "RunningApplication" should "be able to shutdown application" in {
+    val errorMsg = "mock exception"
+    val master = TestProbe()
+    val timeout = Timeout(90, TimeUnit.SECONDS)
+    val application = new RunningApplication(1, master.ref, timeout)
+    Future {
+      application.shutDown()
+    }
+    master.expectMsg(ShutdownApplication(1))
+    master.reply(ShutdownApplicationResult(Success(1)))
+
+    val result = Future {
+      intercept[Exception] {
+        application.shutDown()
+      }
+    }
+    master.expectMsg(ShutdownApplication(1))
+    master.reply(ShutdownApplicationResult(Failure(new Exception(errorMsg))))
+    val exception = Await.result(result, Duration.Inf)
+    assert(exception.getMessage.equals(errorMsg))
+  }
+
+  "RunningApplication" should "be able to ask appmaster" in {
+    val master = TestProbe()
+    val appMaster = TestProbe()
+    val appId = 1
+    val timeout = Timeout(90, TimeUnit.SECONDS)
+    val request = MockAskAppMasterRequest("request")
+    val application = new RunningApplication(appId, master.ref, timeout)
+    val future = application.askAppMaster[MockAskAppMasterResponse](request)
+    master.expectMsg(ResolveAppId(appId))
+    master.reply(ResolveAppIdResult(Success(appMaster.ref)))
+    appMaster.expectMsg(MockAskAppMasterRequest("request"))
+    appMaster.reply(MockAskAppMasterResponse("response"))
+    val result = Await.result(future, Duration.Inf)
+    assert(result.res.equals("response"))
+
+    // ResolveAppId should not be called multiple times
+    val future2 = application.askAppMaster[MockAskAppMasterResponse](request)
+    appMaster.expectMsg(MockAskAppMasterRequest("request"))
+    appMaster.reply(MockAskAppMasterResponse("response"))
+    val result2 = Await.result(future2, Duration.Inf)
+    assert(result2.res.equals("response"))
+  }
+}
+
+object RunningApplicationSpec {
+  case class MockAskAppMasterRequest(req: String)
+
+  case class MockAskAppMasterResponse(res: String)
+}
\ No newline at end of file
diff --git a/daemon/src/test/scala/org/apache/gearpump/cluster/main/MainSpec.scala b/core/src/test/scala/org/apache/gearpump/cluster/main/MainSpec.scala
similarity index 98%
rename from daemon/src/test/scala/org/apache/gearpump/cluster/main/MainSpec.scala
rename to core/src/test/scala/org/apache/gearpump/cluster/main/MainSpec.scala
index 90fdd39..2166976 100644
--- a/daemon/src/test/scala/org/apache/gearpump/cluster/main/MainSpec.scala
+++ b/core/src/test/scala/org/apache/gearpump/cluster/main/MainSpec.scala
@@ -21,23 +21,21 @@
 import java.util.Properties
 
 import akka.testkit.TestProbe
-import org.apache.gearpump.cluster.MasterToWorker.WorkerRegistered
-import org.apache.gearpump.cluster.master.MasterProxy
-import org.apache.gearpump.transport.HostPort
-
-import scala.concurrent.Future
-import scala.util.{Success, Try}
-
-import com.typesafe.config.{ConfigFactory, Config}
-import org.scalatest._
-
+import com.typesafe.config.{Config, ConfigFactory}
 import org.apache.gearpump.cluster.ClientToMaster.{ResolveAppId, ShutdownApplication}
 import org.apache.gearpump.cluster.MasterToAppMaster.{AppMastersDataRequest, ReplayFromTimestampWindowTrailingEdge, _}
 import org.apache.gearpump.cluster.MasterToClient.{ReplayApplicationResult, ResolveAppIdResult, ShutdownApplicationResult}
+import org.apache.gearpump.cluster.MasterToWorker.WorkerRegistered
 import org.apache.gearpump.cluster.WorkerToMaster.RegisterNewWorker
+import org.apache.gearpump.cluster.master.MasterProxy
 import org.apache.gearpump.cluster.{MasterHarness, TestUtil}
+import org.apache.gearpump.transport.HostPort
 import org.apache.gearpump.util.Constants._
 import org.apache.gearpump.util.{Constants, LogUtil, Util}
+import org.scalatest._
+
+import scala.concurrent.Future
+import scala.util.{Success, Try}
 
 class MainSpec extends FlatSpec with Matchers with BeforeAndAfterEach with MasterHarness {
 
diff --git a/daemon/src/test/scala/org/apache/gearpump/cluster/main/MasterWatcherSpec.scala b/core/src/test/scala/org/apache/gearpump/cluster/main/MasterWatcherSpec.scala
similarity index 99%
rename from daemon/src/test/scala/org/apache/gearpump/cluster/main/MasterWatcherSpec.scala
rename to core/src/test/scala/org/apache/gearpump/cluster/main/MasterWatcherSpec.scala
index e1ba8f6..b48fc2a 100644
--- a/daemon/src/test/scala/org/apache/gearpump/cluster/main/MasterWatcherSpec.scala
+++ b/core/src/test/scala/org/apache/gearpump/cluster/main/MasterWatcherSpec.scala
@@ -17,15 +17,14 @@
  */
 package org.apache.gearpump.cluster.main
 
-import scala.concurrent.Await
-import scala.concurrent.duration._
-
 import akka.actor.{ActorSystem, Props}
 import akka.testkit.TestProbe
 import com.typesafe.config.Config
+import org.apache.gearpump.cluster.TestUtil
 import org.scalatest.{FlatSpec, Matchers}
 
-import org.apache.gearpump.cluster.TestUtil
+import scala.concurrent.Await
+import scala.concurrent.duration._
 
 class MasterWatcherSpec extends FlatSpec with Matchers {
   def config: Config = TestUtil.MASTER_CONFIG
diff --git a/daemon/src/test/scala/org/apache/gearpump/cluster/scheduler/PrioritySchedulerSpec.scala b/core/src/test/scala/org/apache/gearpump/cluster/scheduler/PrioritySchedulerSpec.scala
similarity index 99%
rename from daemon/src/test/scala/org/apache/gearpump/cluster/scheduler/PrioritySchedulerSpec.scala
rename to core/src/test/scala/org/apache/gearpump/cluster/scheduler/PrioritySchedulerSpec.scala
index e82dff3..8a3d7d1 100644
--- a/daemon/src/test/scala/org/apache/gearpump/cluster/scheduler/PrioritySchedulerSpec.scala
+++ b/core/src/test/scala/org/apache/gearpump/cluster/scheduler/PrioritySchedulerSpec.scala
@@ -17,14 +17,8 @@
  */
 package org.apache.gearpump.cluster.scheduler
 
-import org.apache.gearpump.cluster.worker.WorkerId
-
-import scala.concurrent.duration._
-
 import akka.actor.{ActorSystem, Props}
 import akka.testkit.{ImplicitSender, TestKit, TestProbe}
-import org.scalatest.{BeforeAndAfterAll, Matchers, WordSpecLike}
-
 import org.apache.gearpump.cluster.AppMasterToMaster.RequestResource
 import org.apache.gearpump.cluster.MasterToAppMaster.ResourceAllocated
 import org.apache.gearpump.cluster.MasterToWorker.{UpdateResourceFailed, WorkerRegistered}
@@ -33,6 +27,10 @@
 import org.apache.gearpump.cluster.master.Master.MasterInfo
 import org.apache.gearpump.cluster.scheduler.Priority.{HIGH, LOW, NORMAL}
 import org.apache.gearpump.cluster.scheduler.Scheduler.ApplicationFinished
+import org.apache.gearpump.cluster.worker.WorkerId
+import org.scalatest.{BeforeAndAfterAll, Matchers, WordSpecLike}
+
+import scala.concurrent.duration._
 
 class PrioritySchedulerSpec(_system: ActorSystem) extends TestKit(_system) with ImplicitSender
   with WordSpecLike with Matchers with BeforeAndAfterAll{
diff --git a/daemon/src/test/scala/org/apache/gearpump/cluster/worker/WorkerSpec.scala b/core/src/test/scala/org/apache/gearpump/cluster/worker/WorkerSpec.scala
similarity index 99%
rename from daemon/src/test/scala/org/apache/gearpump/cluster/worker/WorkerSpec.scala
rename to core/src/test/scala/org/apache/gearpump/cluster/worker/WorkerSpec.scala
index bf25057..e0233f8 100644
--- a/daemon/src/test/scala/org/apache/gearpump/cluster/worker/WorkerSpec.scala
+++ b/core/src/test/scala/org/apache/gearpump/cluster/worker/WorkerSpec.scala
@@ -17,14 +17,9 @@
  */
 package org.apache.gearpump.cluster.worker
 
-import scala.concurrent.Await
-import scala.concurrent.duration._
-
 import akka.actor.{ActorSystem, PoisonPill, Props}
 import akka.testkit.TestProbe
 import com.typesafe.config.{Config, ConfigFactory}
-import org.scalatest._
-
 import org.apache.gearpump.cluster.AppMasterToWorker.{ChangeExecutorResource, LaunchExecutor, ShutdownExecutor}
 import org.apache.gearpump.cluster.MasterToWorker.{UpdateResourceFailed, WorkerRegistered}
 import org.apache.gearpump.cluster.WorkerToAppMaster.{ExecutorLaunchRejected, ShutdownExecutorFailed, ShutdownExecutorSucceed}
@@ -33,6 +28,10 @@
 import org.apache.gearpump.cluster.scheduler.Resource
 import org.apache.gearpump.cluster.{ExecutorJVMConfig, MasterHarness, TestUtil}
 import org.apache.gearpump.util.{ActorSystemBooter, ActorUtil, Constants}
+import org.scalatest._
+
+import scala.concurrent.Await
+import scala.concurrent.duration._
 
 class WorkerSpec extends WordSpec with Matchers with BeforeAndAfterEach with MasterHarness {
   override def config: Config = TestUtil.DEFAULT_CONFIG
diff --git a/core/src/test/scala/org/apache/gearpump/jarstore/FileServerSpec.scala b/core/src/test/scala/org/apache/gearpump/jarstore/FileServerSpec.scala
index c99a031..39b6261 100644
--- a/core/src/test/scala/org/apache/gearpump/jarstore/FileServerSpec.scala
+++ b/core/src/test/scala/org/apache/gearpump/jarstore/FileServerSpec.scala
@@ -22,9 +22,9 @@
 import java.util.concurrent.TimeUnit
 
 import akka.actor.ActorSystem
-import com.typesafe.config.{ConfigValueFactory, ConfigValue}
+import com.google.common.io.Files
+import com.typesafe.config.ConfigValueFactory
 import org.apache.gearpump.cluster.TestUtil
-import org.apache.gearpump.google.common.io.Files
 import org.apache.gearpump.jarstore.local.LocalJarStore
 import org.apache.gearpump.util.{FileUtils, LogUtil}
 import org.scalatest.{BeforeAndAfterAll, Matchers, WordSpecLike}
@@ -39,7 +39,7 @@
   val host = "localhost"
   private val LOG = LogUtil.getLogger(getClass)
 
-  var system: ActorSystem = null
+  var system: ActorSystem = _
 
   override def afterAll {
     if (null != system) {
@@ -75,7 +75,6 @@
 
   "The file server" should {
     "serve the data previously stored" in {
-
       val rootDir = Files.createTempDir()
       val localJarStore: JarStore = new LocalJarStore
       val conf = TestUtil.DEFAULT_CONFIG.withValue("gearpump.jarstore.rootpath",
diff --git a/core/src/test/scala/org/apache/gearpump/metrics/MetricsSpec.scala b/core/src/test/scala/org/apache/gearpump/metrics/MetricsSpec.scala
index 5881640..0855553 100644
--- a/core/src/test/scala/org/apache/gearpump/metrics/MetricsSpec.scala
+++ b/core/src/test/scala/org/apache/gearpump/metrics/MetricsSpec.scala
@@ -18,12 +18,13 @@
 
 package org.apache.gearpump.metrics
 
+import com.codahale.metrics.{Counter => CodaHaleCounter, Histogram => CodaHaleHistogram, Meter => CodaHaleMeter}
+
 import org.mockito.Matchers._
 import org.mockito.Mockito._
 import org.scalatest.mock.MockitoSugar
 import org.scalatest.{FlatSpec, Matchers}
 
-import org.apache.gearpump.codahale.metrics.{Counter => CodaHaleCounter, Histogram => CodaHaleHistogram, Meter => CodaHaleMeter}
 
 class MetricsSpec extends FlatSpec with Matchers with MockitoSugar {
 
diff --git a/core/src/test/scala/org/apache/gearpump/serializer/SerializerSpec.scala b/core/src/test/scala/org/apache/gearpump/serializer/SerializerSpec.scala
index f4bd114..0772a5e 100644
--- a/core/src/test/scala/org/apache/gearpump/serializer/SerializerSpec.scala
+++ b/core/src/test/scala/org/apache/gearpump/serializer/SerializerSpec.scala
@@ -18,19 +18,22 @@
 
 package org.apache.gearpump.serializer
 
+import akka.actor.{ActorSystem, ExtendedActorSystem}
+
+import com.esotericsoftware.kryo.io.{Input, Output}
+import com.esotericsoftware.kryo.{Kryo, Serializer => KryoSerializer}
+import com.typesafe.config.{ConfigFactory, ConfigValueFactory}
+
+import org.apache.gearpump.cluster.TestUtil
+import org.apache.gearpump.serializer.SerializerSpec._
+
+import org.scalatest.mock.MockitoSugar
+import org.scalatest.{FlatSpec, Matchers}
+
 import scala.collection.JavaConverters._
 import scala.concurrent.Await
 import scala.concurrent.duration.Duration
 
-import akka.actor.{ActorSystem, ExtendedActorSystem}
-import com.typesafe.config.{ConfigFactory, ConfigValueFactory}
-import org.scalatest.mock.MockitoSugar
-import org.scalatest.{FlatSpec, Matchers}
-
-import org.apache.gearpump.cluster.TestUtil
-import org.apache.gearpump.esotericsoftware.kryo.io.{Input, Output}
-import org.apache.gearpump.esotericsoftware.kryo.{Kryo, Serializer => KryoSerializer}
-import org.apache.gearpump.serializer.SerializerSpec._
 
 class SerializerSpec extends FlatSpec with Matchers with MockitoSugar {
   val config = ConfigFactory.empty.withValue("gearpump.serializers",
@@ -70,7 +73,7 @@
 
   class ClassASerializer extends KryoSerializer[ClassA] {
     override def write(kryo: Kryo, output: Output, `object`: ClassA): Unit = {
-      output.writeString(classOf[ClassA].getName.toString)
+      output.writeString(classOf[ClassA].getName)
     }
 
     override def read(kryo: Kryo, input: Input, `type`: Class[ClassA]): ClassA = {
diff --git a/core/src/test/scala/org/apache/gearpump/util/FileUtilsSpec.scala b/core/src/test/scala/org/apache/gearpump/util/FileUtilsSpec.scala
index 66abc36..97b35ad 100644
--- a/core/src/test/scala/org/apache/gearpump/util/FileUtilsSpec.scala
+++ b/core/src/test/scala/org/apache/gearpump/util/FileUtilsSpec.scala
@@ -18,13 +18,13 @@
 
 package org.apache.gearpump.util
 
+import com.google.common.io.Files
+
 import java.io.File
 import java.util
 
 import org.scalatest.FlatSpec
 
-import org.apache.gearpump.google.common.io.Files
-
 class FileUtilsSpec extends FlatSpec {
   val TXT =
     """
diff --git a/daemon/src/test/resources/META-INF/services/org.apache.gearpump.jarstore.JarStore b/daemon/src/test/resources/META-INF/services/org.apache.gearpump.jarstore.JarStore
deleted file mode 100644
index e173a8a..0000000
--- a/daemon/src/test/resources/META-INF/services/org.apache.gearpump.jarstore.JarStore
+++ /dev/null
@@ -1,20 +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.
-#
-
-org.apache.gearpump.jarstore.local.LocalJarStore
-org.apache.gearpump.jarstore.dfs.DFSJarStore
\ No newline at end of file
diff --git a/dev-tools/build b/dev-tools/build
new file mode 100755
index 0000000..c43d35f
--- /dev/null
+++ b/dev-tools/build
@@ -0,0 +1,72 @@
+#!/bin/bash
+usage() {
+cat << EOF
+usage: $0 options commands
+OPTIONS:
+  -h      Show this message
+COMMANDS:
+  all:      normal build of all components
+  clean:    clean all sbt artifacts
+  publish:  publish to local
+  publish-m2:  publish to local .m2
+  reset:    remove all idea project files
+  scrub:    remove all untracked git files
+  test:     run coverage and test
+EOF
+}
+while getopts “h” OPTION
+do
+     case "${OPTION}" in
+         h)
+             usage
+             exit 1
+             ;;
+         ?)
+             usage
+             exit
+             ;;
+     esac
+done
+shift $((OPTIND-1))
+if [ $# = 0 ]; then
+  echo sbt assembly pack pack-archive
+  sbt assembly pack pack-archive
+else
+  while (( "$#" )); do
+    case "$1" in 
+      all)
+        echo sbt assembly pack pack-archive
+        sbt assembly pack pack-archive
+        ;;
+      clean)
+        echo rm -rf $(find . -maxdepth 4 -name target -o -name project|grep -v '^./project'|sed '/project\/project/d'|sed '/project\/target/d')
+        rm -rf $(find . -maxdepth 4 -name target -o -name project|grep -v '^./project'|sed '/project\/project/d'|sed '/project\/target/d')
+        rm -rf project/project project/target output
+        ;;
+      publish)
+        echo sbt publish-local
+        sbt publish-local
+        ;;
+      publish-m2)
+        echo sbt publish-m2
+        sbt publish-m2
+        ;;
+      reset)
+        echo rm -rf $(find . -name .idea -type d)
+        rm -rf $(find . -name .idea -type d)
+        ;;
+      scrub)
+        git clean -df
+        ;;
+      test)
+        echo sbt coverage test
+        sbt coverage test
+        ;;
+      *)
+        usage
+        exit
+        ;;
+    esac
+    shift
+  done
+fi
diff --git a/dev-tools/create_apache_bin_release.sh b/dev-tools/create_apache_bin_release.sh
new file mode 100755
index 0000000..b35f98f
--- /dev/null
+++ b/dev-tools/create_apache_bin_release.sh
@@ -0,0 +1,115 @@
+#!/bin/bash
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+# 
+#   http://www.apache.org/licenses/LICENSE-2.0
+# 
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT 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() {
+cat << EOF
+usage: $0 options 
+OPTIONS:
+  -h      Show this message
+  -k      GPG_KEY 
+  -p      GPG_PASSPHRASE
+  -r      RAT Tool
+  -v      Verify signed release
+  -c      Clean before building
+EOF
+}
+VERIFY=false
+RUN_RAT=false
+CLEAN=false
+RELEASE_VERSION=$(grep '^version' version.sbt|sed 's/^.*"\(.*\)"$/\1/')
+GEARPUMP_ARCHIVE_FOLDER=gearpump-2.11-${RELEASE_VERSION}
+GEARPUMP_SCALA_VERSION=gearpump_2.11-${RELEASE_VERSION}
+GEARPUMP_RELEASE_VERSION=${GEARPUMP_SCALA_VERSION}-incubating
+while getopts “hcrvk:p:” OPTION
+do
+     case "${OPTION}" in
+         h)
+             usage
+             exit 1
+             ;;
+         k)
+             GPG_KEY=$OPTARG
+             ;;
+         p)
+             GPG_PASSPHRASE=$OPTARG
+             ;;
+         r)
+             RUN_RAT=true
+             ;;
+         v)
+             VERIFY=true
+             ;;
+         c)
+             CLEAN=true
+             ;;
+         ?)
+             usage
+             exit
+             ;;
+     esac
+done
+shift $((OPTIND-1))
+if [ $VERIFY = "true" ]; then
+  gpg --import KEYS 2>/dev/null
+  echo Verifying ${GEARPUMP_RELEASE_VERSION}-bin.tgz.asc 
+  gpg --verify ${GEARPUMP_RELEASE_VERSION}-bin.tgz.asc
+  exit 0
+fi
+if [ $RUN_RAT = "true" ]; then
+  java -jar ~/rat/trunk/apache-rat/target/apache-rat-0.12-SNAPSHOT.jar -A -f -E ./.rat-excludes -d . | grep '^== File' 
+  exit 0
+fi
+if [ -z $GPG_KEY ]; then
+  echo Missing -k option
+  usage
+  exit 1
+fi
+if [ -z $GPG_PASSPHRASE ]; then
+  echo Missing -p option
+  usage
+  exit 1
+fi
+
+if [ $CLEAN = "true" ]; then
+  dev-tools/build clean reset scrub
+fi
+dev-tools/build all
+PACKED_ARCHIVE=output/target/gearpump-2.11-${RELEASE_VERSION}.tar.gz
+if [ ! -f $PACKED_ARCHIVE ]; then
+  echo "missing $PACKED_ARCHIVE"
+  echo "You must run 'sbt assembly pack pack-archive' first"
+  exit 1
+fi
+mkdir tmp
+cd tmp
+tar xzf ../$PACKED_ARCHIVE
+mv $GEARPUMP_ARCHIVE_FOLDER/* .
+rmdir $GEARPUMP_ARCHIVE_FOLDER
+cp ../NOTICE ../README.md ../CHANGELOG.md ../DISCLAIMER .
+cp ../LICENSE.bin LICENSE
+cp -r ../licenses .
+rsync -a ../tmp/ $GEARPUMP_RELEASE_VERSION 
+tar czf ../${GEARPUMP_RELEASE_VERSION}-bin.tgz $GEARPUMP_RELEASE_VERSION
+echo Signing ../${GEARPUMP_RELEASE_VERSION}-bin.tgz
+echo $GPG_PASSPHRASE | gpg --batch --default-key $GPG_KEY --passphrase-fd 0 --armour --output ../${GEARPUMP_RELEASE_VERSION}-bin.tgz.asc --detach-sig ../${GEARPUMP_RELEASE_VERSION}-bin.tgz
+gpg --print-md MD5 ../${GEARPUMP_RELEASE_VERSION}-bin.tgz > ../${GEARPUMP_RELEASE_VERSION}-bin.tgz.md5
+gpg --print-md SHA1 ../${GEARPUMP_RELEASE_VERSION}-bin.tgz > ../${GEARPUMP_RELEASE_VERSION}-bin.tgz.sha
+cd ..
+rm -rf tmp
+
+
+
diff --git a/dev-tools/create_apache_source_release.sh b/dev-tools/create_apache_source_release.sh
index 5ef2a9f..8961ee9 100755
--- a/dev-tools/create_apache_source_release.sh
+++ b/dev-tools/create_apache_source_release.sh
@@ -77,8 +77,13 @@
   exit 1
 fi
 
+dev-tools/build clean reset scrub
 echo .git > exclude-list
 echo .DS_Store >> exclude-list
+for i in $(ls licenses/*|grep -v LICENSE-jquery.txt|grep -v LICENSE-bootstrap.txt); do
+  echo $i >> exclude-list
+done
+cat exclude-list
 rsync -a --exclude-from exclude-list ../incubator-gearpump/ $GEARPUMP_RELEASE_VERSION
 tar czf ${GEARPUMP_RELEASE_VERSION}-src.tgz $GEARPUMP_RELEASE_VERSION
 echo Signing ${GEARPUMP_RELEASE_VERSION}-src.tgz
diff --git a/dev-tools/dependencies.sh b/dev-tools/dependencies.sh
new file mode 100755
index 0000000..f67ad3a
--- /dev/null
+++ b/dev-tools/dependencies.sh
@@ -0,0 +1,51 @@
+#!/bin/bash
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+# 
+#   http://www.apache.org/licenses/LICENSE-2.0
+# 
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT 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 will generate a file LICENSE.dependencies which can be used as input to the LICENSE.bin (binary release)
+# Some additional work is needed to automate the LICENSE.bin file generation - and it may be difficult to do so.
+# LICENSE.dependencies does save time by categorizing dependencies under the different licenses
+#
+sbt dependencyLicenseInfo | tee dependencyInfo
+cat dependencyInfo | sed -E "s/"$'\E'"\[([0-9]{1,3}((;[0-9]{1,3})*)?)?[m|K]//g" | grep '^\[info\]' | grep -v '^\[info\] Updating'|grep -v '^\[info\] Resolving'|grep -v '^\[info\] Done'|grep -v '^\[info\] Loading '|grep -v '^\[info\] Set ' > licenses.out 
+cat licenses.out | grep '\[info\] [A-Z]' | sed 's/^\[info\] //' | sort | uniq > license.types
+# add a space after 'No license specified'
+sed  -n '/^\[info\] No license specified$/ {
+=
+p
+}' licenses.out | grep -v '^\[info\]' > lines
+cat lines | sort -nr > lines1
+mv lines1 lines
+for i in $(<lines);do
+echo ex -sc "'"${i}'i|[info] '"'" -cx licenses.out > cmd
+sh cmd
+done
+
+rm -f LICENSE.dependencies
+touch LICENSE.dependencies
+cat license.types | while read LINE; do 
+  echo cat licenses.out \| sed "'"'/^\[info\] '$LINE'$/,/^\[info\] $/!d;//d'"'" \| sort \| uniq > cmd
+  echo "$LINE" >> LICENSE.dependencies
+  sh cmd >> LICENSE.dependencies
+  echo ' ' >> LICENSE.dependencies
+done
+cat LICENSE.dependencies | sed 's/^\[info\] //' > LICENSE.d
+mv LICENSE.d LICENSE.dependencies
+
+#cleanup
+rm -f cmd license.types licenses.out lines dependencyInfo
diff --git a/docs/build_doc.sh b/docs/build_doc.sh
index da73dc6..a3e70de 100755
--- a/docs/build_doc.sh
+++ b/docs/build_doc.sh
@@ -1,4 +1,21 @@
 #!/bin/bash
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
 CURDIR=`pwd`
 CURDIRNAME=`basename $CURDIR`
 
@@ -47,8 +64,10 @@
 # render file templates
 echo "Rendering file templates using mustache..."
 TEMP_DIR="tmp"
-rm -rf $TEMP_DIR
-copy_dir docs $TEMP_DIR
+if [ -d "$TEMP_DIR" ]; then
+	rm -rf "$TEMP_DIR"
+fi
+copy_dir contents $TEMP_DIR
 render_files version.yml "$TEMP_DIR/introduction $TEMP_DIR/dev $TEMP_DIR/deployment $TEMP_DIR/api $TEMP_DIR/index.md"
 
 # generate site documents
diff --git a/docs/docs/api/java.md b/docs/contents/api/java.md
similarity index 100%
rename from docs/docs/api/java.md
rename to docs/contents/api/java.md
diff --git a/docs/docs/api/scala.md b/docs/contents/api/scala.md
similarity index 100%
rename from docs/docs/api/scala.md
rename to docs/contents/api/scala.md
diff --git a/docs/docs/deployment/deployment-configuration.md b/docs/contents/deployment/deployment-configuration.md
similarity index 100%
rename from docs/docs/deployment/deployment-configuration.md
rename to docs/contents/deployment/deployment-configuration.md
diff --git a/docs/docs/deployment/deployment-docker.md b/docs/contents/deployment/deployment-docker.md
similarity index 100%
rename from docs/docs/deployment/deployment-docker.md
rename to docs/contents/deployment/deployment-docker.md
diff --git a/docs/docs/deployment/deployment-ha.md b/docs/contents/deployment/deployment-ha.md
similarity index 100%
rename from docs/docs/deployment/deployment-ha.md
rename to docs/contents/deployment/deployment-ha.md
diff --git a/docs/docs/deployment/deployment-local.md b/docs/contents/deployment/deployment-local.md
similarity index 100%
rename from docs/docs/deployment/deployment-local.md
rename to docs/contents/deployment/deployment-local.md
diff --git a/docs/docs/deployment/deployment-msg-delivery.md b/docs/contents/deployment/deployment-msg-delivery.md
similarity index 100%
rename from docs/docs/deployment/deployment-msg-delivery.md
rename to docs/contents/deployment/deployment-msg-delivery.md
diff --git a/docs/docs/deployment/deployment-resource-isolation.md b/docs/contents/deployment/deployment-resource-isolation.md
similarity index 100%
rename from docs/docs/deployment/deployment-resource-isolation.md
rename to docs/contents/deployment/deployment-resource-isolation.md
diff --git a/docs/docs/deployment/deployment-security.md b/docs/contents/deployment/deployment-security.md
similarity index 100%
rename from docs/docs/deployment/deployment-security.md
rename to docs/contents/deployment/deployment-security.md
diff --git a/docs/docs/deployment/deployment-standalone.md b/docs/contents/deployment/deployment-standalone.md
similarity index 100%
rename from docs/docs/deployment/deployment-standalone.md
rename to docs/contents/deployment/deployment-standalone.md
diff --git a/docs/docs/deployment/deployment-ui-authentication.md b/docs/contents/deployment/deployment-ui-authentication.md
similarity index 100%
rename from docs/docs/deployment/deployment-ui-authentication.md
rename to docs/contents/deployment/deployment-ui-authentication.md
diff --git a/docs/docs/deployment/deployment-yarn.md b/docs/contents/deployment/deployment-yarn.md
similarity index 100%
rename from docs/docs/deployment/deployment-yarn.md
rename to docs/contents/deployment/deployment-yarn.md
diff --git a/docs/docs/deployment/get-gearpump-distribution.md b/docs/contents/deployment/get-gearpump-distribution.md
similarity index 100%
rename from docs/docs/deployment/get-gearpump-distribution.md
rename to docs/contents/deployment/get-gearpump-distribution.md
diff --git a/docs/docs/deployment/hardware-requirement.md b/docs/contents/deployment/hardware-requirement.md
similarity index 100%
rename from docs/docs/deployment/hardware-requirement.md
rename to docs/contents/deployment/hardware-requirement.md
diff --git a/docs/docs/dev/dev-connectors.md b/docs/contents/dev/dev-connectors.md
similarity index 100%
rename from docs/docs/dev/dev-connectors.md
rename to docs/contents/dev/dev-connectors.md
diff --git a/docs/docs/dev/dev-custom-serializer.md b/docs/contents/dev/dev-custom-serializer.md
similarity index 100%
rename from docs/docs/dev/dev-custom-serializer.md
rename to docs/contents/dev/dev-custom-serializer.md
diff --git a/docs/docs/dev/dev-ide-setup.md b/docs/contents/dev/dev-ide-setup.md
similarity index 100%
rename from docs/docs/dev/dev-ide-setup.md
rename to docs/contents/dev/dev-ide-setup.md
diff --git a/docs/docs/dev/dev-non-streaming-example.md b/docs/contents/dev/dev-non-streaming-example.md
similarity index 100%
rename from docs/docs/dev/dev-non-streaming-example.md
rename to docs/contents/dev/dev-non-streaming-example.md
diff --git a/docs/docs/dev/dev-rest-api.md b/docs/contents/dev/dev-rest-api.md
similarity index 100%
rename from docs/docs/dev/dev-rest-api.md
rename to docs/contents/dev/dev-rest-api.md
diff --git a/docs/docs/dev/dev-storm.md b/docs/contents/dev/dev-storm.md
similarity index 100%
rename from docs/docs/dev/dev-storm.md
rename to docs/contents/dev/dev-storm.md
diff --git a/docs/docs/dev/dev-write-1st-app.md b/docs/contents/dev/dev-write-1st-app.md
similarity index 100%
rename from docs/docs/dev/dev-write-1st-app.md
rename to docs/contents/dev/dev-write-1st-app.md
diff --git a/docs/docs/img/actor_hierarchy.png b/docs/contents/img/actor_hierarchy.png
similarity index 100%
rename from docs/docs/img/actor_hierarchy.png
rename to docs/contents/img/actor_hierarchy.png
Binary files differ
diff --git a/docs/docs/img/checkpoint_equation.png b/docs/contents/img/checkpoint_equation.png
similarity index 100%
rename from docs/docs/img/checkpoint_equation.png
rename to docs/contents/img/checkpoint_equation.png
Binary files differ
diff --git a/docs/docs/img/checkpoint_interval_equation.png b/docs/contents/img/checkpoint_interval_equation.png
similarity index 100%
rename from docs/docs/img/checkpoint_interval_equation.png
rename to docs/contents/img/checkpoint_interval_equation.png
Binary files differ
diff --git a/docs/docs/img/checkpointing.png b/docs/contents/img/checkpointing.png
similarity index 100%
rename from docs/docs/img/checkpointing.png
rename to docs/contents/img/checkpointing.png
Binary files differ
diff --git a/docs/docs/img/checkpointing_interval.png b/docs/contents/img/checkpointing_interval.png
similarity index 100%
rename from docs/docs/img/checkpointing_interval.png
rename to docs/contents/img/checkpointing_interval.png
Binary files differ
diff --git a/docs/docs/img/clock.png b/docs/contents/img/clock.png
similarity index 100%
rename from docs/docs/img/clock.png
rename to docs/contents/img/clock.png
Binary files differ
diff --git a/docs/docs/img/dag.png b/docs/contents/img/dag.png
similarity index 100%
rename from docs/docs/img/dag.png
rename to docs/contents/img/dag.png
Binary files differ
diff --git a/docs/docs/img/dashboard.gif b/docs/contents/img/dashboard.gif
similarity index 100%
rename from docs/docs/img/dashboard.gif
rename to docs/contents/img/dashboard.gif
Binary files differ
diff --git a/docs/docs/img/dashboard.png b/docs/contents/img/dashboard.png
similarity index 100%
rename from docs/docs/img/dashboard.png
rename to docs/contents/img/dashboard.png
Binary files differ
diff --git a/docs/docs/img/dashboard_3.png b/docs/contents/img/dashboard_3.png
similarity index 100%
rename from docs/docs/img/dashboard_3.png
rename to docs/contents/img/dashboard_3.png
Binary files differ
diff --git a/docs/docs/img/download.jpg b/docs/contents/img/download.jpg
similarity index 100%
rename from docs/docs/img/download.jpg
rename to docs/contents/img/download.jpg
Binary files differ
diff --git a/docs/docs/img/dynamic.png b/docs/contents/img/dynamic.png
similarity index 100%
rename from docs/docs/img/dynamic.png
rename to docs/contents/img/dynamic.png
Binary files differ
diff --git a/docs/docs/img/exact.png b/docs/contents/img/exact.png
similarity index 100%
rename from docs/docs/img/exact.png
rename to docs/contents/img/exact.png
Binary files differ
diff --git a/docs/docs/img/failures.png b/docs/contents/img/failures.png
similarity index 100%
rename from docs/docs/img/failures.png
rename to docs/contents/img/failures.png
Binary files differ
diff --git a/docs/docs/img/flow_control.png b/docs/contents/img/flow_control.png
similarity index 100%
rename from docs/docs/img/flow_control.png
rename to docs/contents/img/flow_control.png
Binary files differ
diff --git a/docs/docs/img/flowcontrol.png b/docs/contents/img/flowcontrol.png
similarity index 100%
rename from docs/docs/img/flowcontrol.png
rename to docs/contents/img/flowcontrol.png
Binary files differ
diff --git a/docs/docs/img/ha.png b/docs/contents/img/ha.png
similarity index 100%
rename from docs/docs/img/ha.png
rename to docs/contents/img/ha.png
Binary files differ
diff --git a/docs/docs/img/kafka_wordcount.png b/docs/contents/img/kafka_wordcount.png
similarity index 100%
rename from docs/docs/img/kafka_wordcount.png
rename to docs/contents/img/kafka_wordcount.png
Binary files differ
diff --git a/docs/docs/img/layout.png b/docs/contents/img/layout.png
similarity index 100%
rename from docs/docs/img/layout.png
rename to docs/contents/img/layout.png
Binary files differ
diff --git a/docs/docs/img/logo.png b/docs/contents/img/logo.png
similarity index 100%
rename from docs/docs/img/logo.png
rename to docs/contents/img/logo.png
Binary files differ
diff --git a/docs/docs/img/logo.svg b/docs/contents/img/logo.svg
similarity index 100%
rename from docs/docs/img/logo.svg
rename to docs/contents/img/logo.svg
diff --git a/docs/docs/img/logo2.png b/docs/contents/img/logo2.png
similarity index 100%
rename from docs/docs/img/logo2.png
rename to docs/contents/img/logo2.png
Binary files differ
diff --git a/docs/docs/img/messageLoss.png b/docs/contents/img/messageLoss.png
similarity index 100%
rename from docs/docs/img/messageLoss.png
rename to docs/contents/img/messageLoss.png
Binary files differ
diff --git a/docs/docs/img/netty_transport.png b/docs/contents/img/netty_transport.png
similarity index 100%
rename from docs/docs/img/netty_transport.png
rename to docs/contents/img/netty_transport.png
Binary files differ
diff --git a/docs/docs/img/replay.png b/docs/contents/img/replay.png
similarity index 100%
rename from docs/docs/img/replay.png
rename to docs/contents/img/replay.png
Binary files differ
diff --git a/docs/docs/img/shuffle.png b/docs/contents/img/shuffle.png
similarity index 100%
rename from docs/docs/img/shuffle.png
rename to docs/contents/img/shuffle.png
Binary files differ
diff --git a/docs/docs/img/storm_gearpump_cluster.png b/docs/contents/img/storm_gearpump_cluster.png
similarity index 100%
rename from docs/docs/img/storm_gearpump_cluster.png
rename to docs/contents/img/storm_gearpump_cluster.png
Binary files differ
diff --git a/docs/docs/img/storm_gearpump_dag.png b/docs/contents/img/storm_gearpump_dag.png
similarity index 100%
rename from docs/docs/img/storm_gearpump_dag.png
rename to docs/contents/img/storm_gearpump_dag.png
Binary files differ
diff --git a/docs/docs/img/submit.png b/docs/contents/img/submit.png
similarity index 100%
rename from docs/docs/img/submit.png
rename to docs/contents/img/submit.png
Binary files differ
diff --git a/docs/docs/img/submit2.png b/docs/contents/img/submit2.png
similarity index 100%
rename from docs/docs/img/submit2.png
rename to docs/contents/img/submit2.png
Binary files differ
diff --git a/docs/docs/img/through_vs_message_size.png b/docs/contents/img/through_vs_message_size.png
similarity index 100%
rename from docs/docs/img/through_vs_message_size.png
rename to docs/contents/img/through_vs_message_size.png
Binary files differ
diff --git a/docs/docs/index.md b/docs/contents/index.md
similarity index 100%
rename from docs/docs/index.md
rename to docs/contents/index.md
diff --git a/docs/docs/introduction/basic-concepts.md b/docs/contents/introduction/basic-concepts.md
similarity index 100%
rename from docs/docs/introduction/basic-concepts.md
rename to docs/contents/introduction/basic-concepts.md
diff --git a/docs/docs/introduction/commandline.md b/docs/contents/introduction/commandline.md
similarity index 100%
rename from docs/docs/introduction/commandline.md
rename to docs/contents/introduction/commandline.md
diff --git a/docs/docs/introduction/features.md b/docs/contents/introduction/features.md
similarity index 100%
rename from docs/docs/introduction/features.md
rename to docs/contents/introduction/features.md
diff --git a/docs/docs/introduction/gearpump-internals.md b/docs/contents/introduction/gearpump-internals.md
similarity index 100%
rename from docs/docs/introduction/gearpump-internals.md
rename to docs/contents/introduction/gearpump-internals.md
diff --git a/docs/docs/introduction/message-delivery.md b/docs/contents/introduction/message-delivery.md
similarity index 100%
rename from docs/docs/introduction/message-delivery.md
rename to docs/contents/introduction/message-delivery.md
diff --git a/docs/docs/introduction/performance-report.md b/docs/contents/introduction/performance-report.md
similarity index 100%
rename from docs/docs/introduction/performance-report.md
rename to docs/contents/introduction/performance-report.md
diff --git a/docs/docs/introduction/submit-your-1st-application.md b/docs/contents/introduction/submit-your-1st-application.md
similarity index 100%
rename from docs/docs/introduction/submit-your-1st-application.md
rename to docs/contents/introduction/submit-your-1st-application.md
diff --git a/docs/mkdocs.yml b/docs/mkdocs.yml
index a769960..6e23e41 100644
--- a/docs/mkdocs.yml
+++ b/docs/mkdocs.yml
@@ -2,6 +2,8 @@
 
 repo_url: 'https://github.com/apache/incubator-gearpump'
 
+edit_uri: 'edit/master/docs/contents'
+
 use_directory_urls: false
 
 # files under docs/ are copied to tmp/
diff --git a/docs/version.yml b/docs/version.yml
index b13cd9d..8a18936 100644
--- a/docs/version.yml
+++ b/docs/version.yml
@@ -1,5 +1,5 @@
 ---
-GEARPUMP_VERSION: "0.8.2-SNAPSHOT"
+GEARPUMP_VERSION: "0.8.3-SNAPSHOT"
 SCALA_BINARY_VERSION: "2.11"
 SCALA_VERSION: "2.11.8"
 ---
diff --git a/examples/distributedshell/src/main/scala/org/apache/gearpump/examples/distributedshell/DistributedShell.scala b/examples/distributedshell/src/main/scala/org/apache/gearpump/examples/distributedshell/DistributedShell.scala
index c4eec07..6db8531 100644
--- a/examples/distributedshell/src/main/scala/org/apache/gearpump/examples/distributedshell/DistributedShell.scala
+++ b/examples/distributedshell/src/main/scala/org/apache/gearpump/examples/distributedshell/DistributedShell.scala
@@ -33,9 +33,9 @@
   override def main(akkaConf: Config, args: Array[String]): Unit = {
     LOG.info(s"Distributed shell submitting application...")
     val context = ClientContext(akkaConf)
-    val appId = context.submit(Application[DistShellAppMaster]("DistributedShell",
+    val app = context.submit(Application[DistShellAppMaster]("DistributedShell",
     UserConfig.empty))
     context.close()
-    LOG.info(s"Distributed Shell Application started with appId $appId !")
+    LOG.info(s"Distributed Shell Application started with appId ${app.appId} !")
   }
 }
diff --git a/examples/distributeservice/src/main/scala/org/apache/gearpump/experiments/distributeservice/DistributeService.scala b/examples/distributeservice/src/main/scala/org/apache/gearpump/experiments/distributeservice/DistributeService.scala
index df7a517..655389b 100644
--- a/examples/distributeservice/src/main/scala/org/apache/gearpump/experiments/distributeservice/DistributeService.scala
+++ b/examples/distributeservice/src/main/scala/org/apache/gearpump/experiments/distributeservice/DistributeService.scala
@@ -33,9 +33,9 @@
   override def main(akkaConf: Config, args: Array[String]): Unit = {
     LOG.info(s"Distribute Service submitting application...")
     val context = ClientContext(akkaConf)
-    val appId = context.submit(Application[DistServiceAppMaster]("DistributedService",
+    val app = context.submit(Application[DistServiceAppMaster]("DistributedService",
       UserConfig.empty))
     context.close()
-    LOG.info(s"Distribute Service Application started with appId $appId !")
+    LOG.info(s"Distribute Service Application started with appId ${app.appId} !")
   }
 }
diff --git a/examples/pagerank/src/main/scala/org/apache/gearpump/experiments/pagerank/PageRankApplication.scala b/examples/pagerank/src/main/scala/org/apache/gearpump/experiments/pagerank/PageRankApplication.scala
index 023ee35..c7bfb43 100644
--- a/examples/pagerank/src/main/scala/org/apache/gearpump/experiments/pagerank/PageRankApplication.scala
+++ b/examples/pagerank/src/main/scala/org/apache/gearpump/experiments/pagerank/PageRankApplication.scala
@@ -21,7 +21,7 @@
 
 import org.apache.gearpump.cluster.{Application, ApplicationMaster, UserConfig}
 import org.apache.gearpump.experiments.pagerank.PageRankApplication.NodeWithTaskId
-import org.apache.gearpump.partitioner.HashPartitioner
+import org.apache.gearpump.streaming.partitioner.HashPartitioner
 import org.apache.gearpump.streaming.appmaster.AppMaster
 import org.apache.gearpump.streaming.{Processor, StreamApplication}
 import org.apache.gearpump.util.Graph
diff --git a/examples/streaming/complexdag/src/main/scala/org/apache/gearpump/streaming/examples/complexdag/Dag.scala b/examples/streaming/complexdag/src/main/scala/org/apache/gearpump/streaming/examples/complexdag/Dag.scala
index 3b6ceb8..165df62 100644
--- a/examples/streaming/complexdag/src/main/scala/org/apache/gearpump/streaming/examples/complexdag/Dag.scala
+++ b/examples/streaming/complexdag/src/main/scala/org/apache/gearpump/streaming/examples/complexdag/Dag.scala
@@ -23,7 +23,7 @@
 import org.apache.gearpump.cluster.UserConfig
 import org.apache.gearpump.cluster.client.ClientContext
 import org.apache.gearpump.cluster.main.{ArgumentsParser, CLIOption, ParseResult}
-import org.apache.gearpump.partitioner.HashPartitioner
+import org.apache.gearpump.streaming.partitioner.HashPartitioner
 import org.apache.gearpump.streaming.task.TaskContext
 import org.apache.gearpump.streaming.{Processor, StreamApplication}
 import org.apache.gearpump.util.Graph.{Node => GraphNode}
diff --git a/examples/streaming/fsio/src/main/scala/org/apache/gearpump/streaming/examples/fsio/SequenceFileIO.scala b/examples/streaming/fsio/src/main/scala/org/apache/gearpump/streaming/examples/fsio/SequenceFileIO.scala
index 5c75904..3a80549 100644
--- a/examples/streaming/fsio/src/main/scala/org/apache/gearpump/streaming/examples/fsio/SequenceFileIO.scala
+++ b/examples/streaming/fsio/src/main/scala/org/apache/gearpump/streaming/examples/fsio/SequenceFileIO.scala
@@ -23,7 +23,7 @@
 import org.apache.gearpump.cluster.UserConfig
 import org.apache.gearpump.cluster.client.ClientContext
 import org.apache.gearpump.cluster.main.{ArgumentsParser, CLIOption, ParseResult}
-import org.apache.gearpump.partitioner.ShufflePartitioner
+import org.apache.gearpump.streaming.partitioner.ShufflePartitioner
 import org.apache.gearpump.streaming.examples.fsio.HadoopConfig._
 import org.apache.gearpump.streaming.{Processor, StreamApplication}
 import org.apache.gearpump.util.Graph._
diff --git a/examples/streaming/hbase/src/main/scala/org/apache/gearpump/streaming/examples/hbase/HBaseConn.scala b/examples/streaming/hbase/src/main/scala/org/apache/gearpump/streaming/examples/hbase/HBaseConn.scala
new file mode 100644
index 0000000..df8c2f5
--- /dev/null
+++ b/examples/streaming/hbase/src/main/scala/org/apache/gearpump/streaming/examples/hbase/HBaseConn.scala
@@ -0,0 +1,67 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES 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.gearpump.streaming.examples.hbase
+
+import akka.actor.ActorSystem
+import org.apache.gearpump.cluster.UserConfig
+import org.apache.gearpump.cluster.client.ClientContext
+import org.apache.gearpump.cluster.main.{ArgumentsParser, CLIOption, ParseResult}
+import org.apache.gearpump.external.hbase.HBaseSink
+import org.apache.gearpump.streaming.StreamApplication
+import org.apache.gearpump.streaming.partitioner.HashPartitioner
+import org.apache.gearpump.streaming.sink.DataSinkProcessor
+import org.apache.gearpump.streaming.source.DataSourceProcessor
+import org.apache.gearpump.util.Graph.Node
+import org.apache.gearpump.util.{AkkaApp, Graph, LogUtil}
+import org.slf4j.Logger
+
+object HBaseConn extends AkkaApp with ArgumentsParser {
+  private val LOG: Logger = LogUtil.getLogger(getClass)
+  val RUN_FOR_EVER = -1
+
+  override val options: Array[(String, CLIOption[Any])] = Array(
+    "splitNum" -> CLIOption[Int]("<how many sum tasks>", required = false, defaultValue = Some(1)),
+    "sinkNum" -> CLIOption[Int]("<how many sum tasks>", required = false, defaultValue = Some(1))
+  )
+
+  def application(config: ParseResult, system: ActorSystem): StreamApplication = {
+    implicit val actorSystem = system
+
+    val splitNum = config.getInt("splitNum")
+    val sinkNum = config.getInt("sinkNum")
+
+    val split = new Split
+    val sourceProcessor = DataSourceProcessor(split, splitNum, "Split")
+    val sink = HBaseSink(UserConfig.empty, "hbase")
+    val sinkProcessor = DataSinkProcessor(sink, sinkNum)
+    val partitioner = new HashPartitioner
+    val computation = sourceProcessor ~ partitioner ~> sinkProcessor
+    val application = StreamApplication("HBase", Graph(computation), UserConfig.empty)
+
+    application
+
+  }
+
+  override def main(akkaConf: Config, args: Array[String]): Unit = {
+    val config = parse(args)
+    val context = ClientContext(akkaConf)
+    val appId = context.submit(application(config, context.system))
+    context.close()
+  }
+}
diff --git a/examples/streaming/hbase/src/main/scala/org/apache/gearpump/streaming/examples/hbase/Split.scala b/examples/streaming/hbase/src/main/scala/org/apache/gearpump/streaming/examples/hbase/Split.scala
new file mode 100644
index 0000000..d16cd87
--- /dev/null
+++ b/examples/streaming/hbase/src/main/scala/org/apache/gearpump/streaming/examples/hbase/Split.scala
@@ -0,0 +1,47 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES 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.gearpump.streaming.examples.hbase
+
+import java.time.Instant
+
+import org.apache.gearpump.Message
+import org.apache.gearpump.streaming.source.DataSource
+import org.apache.gearpump.streaming.task.TaskContext
+import org.apache.hadoop.hbase.util.Bytes
+
+class Split extends DataSource {
+
+  private var x = 0
+
+  override def open(context: TaskContext, startTime: Instant): Unit = {}
+
+  override def read(): Message = {
+
+    val tuple = (Bytes.toBytes(s"$x"), Bytes.toBytes("group"),
+      Bytes.toBytes("group:name"), Bytes.toBytes("99"))
+    x+=1
+
+    Message(tuple)
+  }
+
+  override def close(): Unit = {}
+
+  override def getWatermark: Instant = Instant.now()
+
+}
diff --git a/examples/streaming/kafka/src/main/scala/org/apache/gearpump/streaming/examples/kafka/KafkaReadWrite.scala b/examples/streaming/kafka/src/main/scala/org/apache/gearpump/streaming/examples/kafka/KafkaReadWrite.scala
index cfeef5b..4b48e7d 100644
--- a/examples/streaming/kafka/src/main/scala/org/apache/gearpump/streaming/examples/kafka/KafkaReadWrite.scala
+++ b/examples/streaming/kafka/src/main/scala/org/apache/gearpump/streaming/examples/kafka/KafkaReadWrite.scala
@@ -27,7 +27,7 @@
 import org.apache.gearpump.cluster.UserConfig
 import org.apache.gearpump.cluster.client.ClientContext
 import org.apache.gearpump.cluster.main.{ArgumentsParser, CLIOption, ParseResult}
-import org.apache.gearpump.partitioner.ShufflePartitioner
+import org.apache.gearpump.streaming.partitioner.ShufflePartitioner
 import org.apache.gearpump.streaming.StreamApplication
 import org.apache.gearpump.streaming.kafka._
 import org.apache.gearpump.streaming.sink.DataSinkProcessor
diff --git a/examples/streaming/kafka/src/main/scala/org/apache/gearpump/streaming/examples/kafka/dsl/KafkaReadWrite.scala b/examples/streaming/kafka/src/main/scala/org/apache/gearpump/streaming/examples/kafka/dsl/KafkaReadWrite.scala
index 49d3619..cbfe57a 100644
--- a/examples/streaming/kafka/src/main/scala/org/apache/gearpump/streaming/examples/kafka/dsl/KafkaReadWrite.scala
+++ b/examples/streaming/kafka/src/main/scala/org/apache/gearpump/streaming/examples/kafka/dsl/KafkaReadWrite.scala
@@ -21,8 +21,8 @@
 import java.util.Properties
 
 import org.apache.gearpump.cluster.client.ClientContext
-import org.apache.gearpump.cluster.main.{CLIOption, ArgumentsParser}
-import org.apache.gearpump.streaming.dsl.StreamApp
+import org.apache.gearpump.cluster.main.{ArgumentsParser, CLIOption}
+import org.apache.gearpump.streaming.dsl.scalaapi.StreamApp
 import org.apache.gearpump.streaming.kafka.KafkaStoreFactory
 import org.apache.gearpump.streaming.kafka.dsl.KafkaDSL
 import org.apache.gearpump.streaming.kafka.dsl.KafkaDSL._
diff --git a/examples/streaming/kafka/src/main/scala/org/apache/gearpump/streaming/examples/kafka/wordcount/KafkaWordCount.scala b/examples/streaming/kafka/src/main/scala/org/apache/gearpump/streaming/examples/kafka/wordcount/KafkaWordCount.scala
index aa9842f..80f0ff7 100644
--- a/examples/streaming/kafka/src/main/scala/org/apache/gearpump/streaming/examples/kafka/wordcount/KafkaWordCount.scala
+++ b/examples/streaming/kafka/src/main/scala/org/apache/gearpump/streaming/examples/kafka/wordcount/KafkaWordCount.scala
@@ -28,7 +28,7 @@
 import org.apache.gearpump.cluster.UserConfig
 import org.apache.gearpump.cluster.client.ClientContext
 import org.apache.gearpump.cluster.main.{ArgumentsParser, CLIOption, ParseResult}
-import org.apache.gearpump.partitioner.HashPartitioner
+import org.apache.gearpump.streaming.partitioner.HashPartitioner
 import org.apache.gearpump.streaming.kafka._
 import org.apache.gearpump.streaming.sink.DataSinkProcessor
 import org.apache.gearpump.streaming.source.DataSourceProcessor
diff --git a/examples/streaming/sol/src/main/scala/org/apache/gearpump/streaming/examples/sol/SOL.scala b/examples/streaming/sol/src/main/scala/org/apache/gearpump/streaming/examples/sol/SOL.scala
index fb80ad3..01aa95e 100644
--- a/examples/streaming/sol/src/main/scala/org/apache/gearpump/streaming/examples/sol/SOL.scala
+++ b/examples/streaming/sol/src/main/scala/org/apache/gearpump/streaming/examples/sol/SOL.scala
@@ -23,7 +23,7 @@
 import org.apache.gearpump.cluster.UserConfig
 import org.apache.gearpump.cluster.client.ClientContext
 import org.apache.gearpump.cluster.main.{ArgumentsParser, CLIOption, ParseResult}
-import org.apache.gearpump.partitioner.ShufflePartitioner
+import org.apache.gearpump.streaming.partitioner.ShufflePartitioner
 import org.apache.gearpump.streaming.{Processor, StreamApplication}
 import org.apache.gearpump.util.Graph._
 import org.apache.gearpump.util.{AkkaApp, Graph, LogUtil}
diff --git a/examples/streaming/state/src/main/scala/org/apache/gearpump/streaming/examples/state/MessageCountApp.scala b/examples/streaming/state/src/main/scala/org/apache/gearpump/streaming/examples/state/MessageCountApp.scala
index 9bd2bc5..59289a5 100644
--- a/examples/streaming/state/src/main/scala/org/apache/gearpump/streaming/examples/state/MessageCountApp.scala
+++ b/examples/streaming/state/src/main/scala/org/apache/gearpump/streaming/examples/state/MessageCountApp.scala
@@ -27,7 +27,7 @@
 import org.apache.gearpump.cluster.UserConfig
 import org.apache.gearpump.cluster.client.ClientContext
 import org.apache.gearpump.cluster.main.{ArgumentsParser, CLIOption, ParseResult}
-import org.apache.gearpump.partitioner.HashPartitioner
+import org.apache.gearpump.streaming.partitioner.HashPartitioner
 import org.apache.gearpump.streaming.examples.state.processor.CountProcessor
 import org.apache.gearpump.streaming.hadoop.HadoopCheckpointStoreFactory
 import org.apache.gearpump.streaming.hadoop.lib.rotation.FileSizeRotation
diff --git a/examples/streaming/state/src/main/scala/org/apache/gearpump/streaming/examples/state/WindowAverageApp.scala b/examples/streaming/state/src/main/scala/org/apache/gearpump/streaming/examples/state/WindowAverageApp.scala
index 629deb7..50235bc 100644
--- a/examples/streaming/state/src/main/scala/org/apache/gearpump/streaming/examples/state/WindowAverageApp.scala
+++ b/examples/streaming/state/src/main/scala/org/apache/gearpump/streaming/examples/state/WindowAverageApp.scala
@@ -24,7 +24,7 @@
 import org.apache.gearpump.cluster.UserConfig
 import org.apache.gearpump.cluster.client.ClientContext
 import org.apache.gearpump.cluster.main.{ArgumentsParser, CLIOption, ParseResult}
-import org.apache.gearpump.partitioner.HashPartitioner
+import org.apache.gearpump.streaming.partitioner.HashPartitioner
 import org.apache.gearpump.streaming.examples.state.processor.{NumberGeneratorProcessor, WindowAverageProcessor}
 import org.apache.gearpump.streaming.hadoop.HadoopCheckpointStoreFactory
 import org.apache.gearpump.streaming.state.impl.{PersistentStateConfig, WindowConfig}
diff --git a/examples/streaming/wordcount-java/src/main/java/org/apache/gearpump/streaming/examples/wordcountjava/WordCount.java b/examples/streaming/wordcount-java/src/main/java/org/apache/gearpump/streaming/examples/wordcountjava/WordCount.java
index 6b5bba0..5e3d472 100644
--- a/examples/streaming/wordcount-java/src/main/java/org/apache/gearpump/streaming/examples/wordcountjava/WordCount.java
+++ b/examples/streaming/wordcount-java/src/main/java/org/apache/gearpump/streaming/examples/wordcountjava/WordCount.java
@@ -23,12 +23,11 @@
 import org.apache.gearpump.cluster.UserConfig;
 import org.apache.gearpump.cluster.client.ClientContext;
 import org.apache.gearpump.cluster.embedded.EmbeddedCluster;
-import org.apache.gearpump.partitioner.HashPartitioner;
-import org.apache.gearpump.partitioner.Partitioner;
+import org.apache.gearpump.streaming.partitioner.HashPartitioner;
+import org.apache.gearpump.streaming.partitioner.Partitioner;
 import org.apache.gearpump.streaming.javaapi.Graph;
 import org.apache.gearpump.streaming.javaapi.Processor;
 import org.apache.gearpump.streaming.javaapi.StreamApplication;
-import org.apache.gearpump.util.Constants;
 
 /** Java version of WordCount with Processor Graph API */
 public class WordCount {
diff --git a/examples/streaming/wordcount-java/src/main/java/org/apache/gearpump/streaming/examples/wordcountjava/dsl/WordCount.java b/examples/streaming/wordcount-java/src/main/java/org/apache/gearpump/streaming/examples/wordcountjava/dsl/WordCount.java
index 0ecc42e..2942861 100644
--- a/examples/streaming/wordcount-java/src/main/java/org/apache/gearpump/streaming/examples/wordcountjava/dsl/WordCount.java
+++ b/examples/streaming/wordcount-java/src/main/java/org/apache/gearpump/streaming/examples/wordcountjava/dsl/WordCount.java
@@ -19,21 +19,23 @@
 package org.apache.gearpump.streaming.examples.wordcountjava.dsl;
 
 import com.typesafe.config.Config;
+import org.apache.gearpump.Message;
 import org.apache.gearpump.cluster.ClusterConfig;
 import org.apache.gearpump.cluster.UserConfig;
 import org.apache.gearpump.cluster.client.ClientContext;
 import org.apache.gearpump.streaming.dsl.javaapi.JavaStream;
 import org.apache.gearpump.streaming.dsl.javaapi.JavaStreamApp;
-import org.apache.gearpump.streaming.javaapi.dsl.functions.FlatMapFunction;
-import org.apache.gearpump.streaming.javaapi.dsl.functions.GroupByFunction;
-import org.apache.gearpump.streaming.javaapi.dsl.functions.MapFunction;
-import org.apache.gearpump.streaming.javaapi.dsl.functions.ReduceFunction;
+import org.apache.gearpump.streaming.dsl.api.functions.MapFunction;
+import org.apache.gearpump.streaming.dsl.api.functions.ReduceFunction;
+import org.apache.gearpump.streaming.dsl.javaapi.functions.FlatMapFunction;
+import org.apache.gearpump.streaming.dsl.javaapi.functions.GroupByFunction;
+import org.apache.gearpump.streaming.source.DataSource;
+import org.apache.gearpump.streaming.task.TaskContext;
 import scala.Tuple2;
 
-import java.util.ArrayList;
+import java.time.Instant;
 import java.util.Arrays;
 import java.util.Iterator;
-import java.util.List;
 
 /** Java version of WordCount with high level DSL API */
 public class WordCount {
@@ -45,41 +47,80 @@
   public static void main(Config akkaConf, String[] args) throws InterruptedException {
     ClientContext context = new ClientContext(akkaConf);
     JavaStreamApp app = new JavaStreamApp("JavaDSL", context, UserConfig.empty());
-    List<String> source = new ArrayList<>(Arrays.asList("This is a good start, bingo!! bingo!!"));
 
-    JavaStream<String> sentence = app.source(source, 1, UserConfig.empty(), "source");
+    JavaStream<String> sentence = app.source(new StringSource("This is a good start, bingo!! bingo!!"),
+        1, UserConfig.empty(), "source");
 
-    JavaStream<String> words = sentence.flatMap(new FlatMapFunction<String, String>() {
-      @Override
-      public Iterator<String> apply(String s) {
-        return new ArrayList<String>(Arrays.asList(s.split("\\s+"))).iterator();
-      }
-    }, "flatMap");
+    JavaStream<String> words = sentence.flatMap(new Split(), "flatMap");
 
-    JavaStream<Tuple2<String, Integer>> ones = words.map(new MapFunction<String, Tuple2<String, Integer>>() {
-      @Override
-      public Tuple2<String, Integer> apply(String s) {
-        return new Tuple2<String, Integer>(s, 1);
-      }
-    }, "map");
+    JavaStream<Tuple2<String, Integer>> ones = words.map(new Ones(), "map");
 
-    JavaStream<Tuple2<String, Integer>> groupedOnes = ones.groupBy(new GroupByFunction<Tuple2<String, Integer>, String>() {
-      @Override
-      public String apply(Tuple2<String, Integer> tuple) {
-        return tuple._1();
-      }
-    }, 1, "groupBy");
+    JavaStream<Tuple2<String, Integer>> groupedOnes = ones.groupBy(new TupleKey(), 1, "groupBy");
 
-    JavaStream<Tuple2<String, Integer>> wordcount = groupedOnes.reduce(new ReduceFunction<Tuple2<String, Integer>>() {
-      @Override
-      public Tuple2<String, Integer> apply(Tuple2<String, Integer> t1, Tuple2<String, Integer> t2) {
-        return new Tuple2<String, Integer>(t1._1(), t1._2() + t2._2());
-      }
-    }, "reduce");
+    JavaStream<Tuple2<String, Integer>> wordcount = groupedOnes.reduce(new Count(), "reduce");
 
     wordcount.log();
 
-    app.run();
+    app.submit();
     context.close();
   }
+
+  private static class StringSource implements DataSource {
+
+    private final String str;
+
+    StringSource(String str) {
+      this.str = str;
+    }
+
+    @Override
+    public void open(TaskContext context, Instant startTime) {
+    }
+
+    @Override
+    public Message read() {
+      return Message.apply(str, Instant.now().toEpochMilli());
+    }
+
+    @Override
+    public void close() {
+    }
+
+    @Override
+    public Instant getWatermark() {
+      return Instant.now();
+    }
+  }
+
+  private static class Split extends FlatMapFunction<String, String> {
+
+    @Override
+    public Iterator<String> apply(String s) {
+      return Arrays.asList(s.split("\\s+")).iterator();
+    }
+  }
+
+  private static class Ones extends MapFunction<String, Tuple2<String, Integer>> {
+
+    @Override
+    public Tuple2<String, Integer> apply(String s) {
+      return new Tuple2<>(s, 1);
+    }
+  }
+
+  private static class Count extends ReduceFunction<Tuple2<String, Integer>> {
+
+    @Override
+    public Tuple2<String, Integer> apply(Tuple2<String, Integer> t1, Tuple2<String, Integer> t2) {
+      return new Tuple2<>(t1._1(), t1._2() + t2._2());
+    }
+  }
+
+  private static class TupleKey extends GroupByFunction<Tuple2<String, Integer>, String> {
+
+    @Override
+    public String apply(Tuple2<String, Integer> tuple) {
+      return tuple._1();
+    }
+  }
 }
diff --git a/examples/streaming/wordcount/src/main/scala/org/apache/gearpump/streaming/examples/wordcount/WordCount.scala b/examples/streaming/wordcount/src/main/scala/org/apache/gearpump/streaming/examples/wordcount/WordCount.scala
index 9580e63..0e3d840 100644
--- a/examples/streaming/wordcount/src/main/scala/org/apache/gearpump/streaming/examples/wordcount/WordCount.scala
+++ b/examples/streaming/wordcount/src/main/scala/org/apache/gearpump/streaming/examples/wordcount/WordCount.scala
@@ -23,7 +23,7 @@
 import org.apache.gearpump.cluster.client.ClientContext
 import org.apache.gearpump.cluster.embedded.EmbeddedCluster
 import org.apache.gearpump.cluster.main.{ArgumentsParser, CLIOption, ParseResult}
-import org.apache.gearpump.partitioner.HashPartitioner
+import org.apache.gearpump.streaming.partitioner.HashPartitioner
 import org.apache.gearpump.streaming.source.DataSourceProcessor
 import org.apache.gearpump.streaming.{Processor, StreamApplication}
 import org.apache.gearpump.util.Graph.Node
diff --git a/examples/streaming/wordcount/src/main/scala/org/apache/gearpump/streaming/examples/wordcount/dsl/WindowedWordCount.scala b/examples/streaming/wordcount/src/main/scala/org/apache/gearpump/streaming/examples/wordcount/dsl/WindowedWordCount.scala
index 4f43fd4..401eac0 100644
--- a/examples/streaming/wordcount/src/main/scala/org/apache/gearpump/streaming/examples/wordcount/dsl/WindowedWordCount.scala
+++ b/examples/streaming/wordcount/src/main/scala/org/apache/gearpump/streaming/examples/wordcount/dsl/WindowedWordCount.scala
@@ -22,7 +22,7 @@
 import org.apache.gearpump.Message
 import org.apache.gearpump.cluster.client.ClientContext
 import org.apache.gearpump.cluster.main.{ArgumentsParser, CLIOption}
-import org.apache.gearpump.streaming.dsl.{LoggerSink, StreamApp}
+import org.apache.gearpump.streaming.dsl.scalaapi.{LoggerSink, StreamApp}
 import org.apache.gearpump.streaming.dsl.window.api.{EventTimeTrigger, FixedWindow}
 import org.apache.gearpump.streaming.source.DataSource
 import org.apache.gearpump.streaming.task.TaskContext
diff --git a/examples/streaming/wordcount/src/main/scala/org/apache/gearpump/streaming/examples/wordcount/dsl/WordCount.scala b/examples/streaming/wordcount/src/main/scala/org/apache/gearpump/streaming/examples/wordcount/dsl/WordCount.scala
index 22f597c..1cbfb22 100644
--- a/examples/streaming/wordcount/src/main/scala/org/apache/gearpump/streaming/examples/wordcount/dsl/WordCount.scala
+++ b/examples/streaming/wordcount/src/main/scala/org/apache/gearpump/streaming/examples/wordcount/dsl/WordCount.scala
@@ -20,8 +20,8 @@
 
 import org.apache.gearpump.cluster.client.ClientContext
 import org.apache.gearpump.cluster.main.{ArgumentsParser, CLIOption}
-import org.apache.gearpump.streaming.dsl.StreamApp
-import org.apache.gearpump.streaming.dsl.StreamApp._
+import org.apache.gearpump.streaming.dsl.scalaapi.StreamApp
+import org.apache.gearpump.streaming.dsl.scalaapi.StreamApp._
 import org.apache.gearpump.util.AkkaApp
 
 /** Same WordCount with High level DSL syntax */
diff --git a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/GearAttributes.scala b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/GearAttributes.scala
index 016a7b2..4384b39 100644
--- a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/GearAttributes.scala
+++ b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/GearAttributes.scala
@@ -7,7 +7,7 @@
  * "License"); you may not use this 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,
diff --git a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/GearpumpMaterializer.scala b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/GearpumpMaterializer.scala
index 9ff701c..07c95f8 100644
--- a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/GearpumpMaterializer.scala
+++ b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/GearpumpMaterializer.scala
@@ -7,7 +7,7 @@
  * "License"); you may not use this 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,
@@ -24,11 +24,11 @@
 import akka.actor.{ActorContext, ActorRef, ActorRefFactory, ActorSystem, Cancellable, ExtendedActorSystem}
 import akka.event.{Logging, LoggingAdapter}
 import akka.stream.Attributes.Attribute
-import akka.stream._
 import akka.stream.impl.Stages.SymbolicGraphStage
-import akka.stream.impl.StreamLayout._
-import akka.stream.impl._
+import akka.stream.impl.StreamLayout.{Atomic, Combine, CopiedModule, Ignore, MaterializedValueNode, Module, Transform}
+import akka.stream.{ActorAttributes, ActorMaterializerSettings, Attributes, ClosedShape, Fusing, Graph, InPort, OutPort, SinkShape}
 import akka.stream.impl.fusing.{GraphInterpreterShell, GraphStageModule}
+import akka.stream.impl.{ExtendedActorMaterializer, StreamSupervisor}
 import akka.stream.stage.GraphStage
 import org.apache.gearpump.akkastream.GearpumpMaterializer.Edge
 import org.apache.gearpump.akkastream.graph.GraphPartitioner.Strategy
@@ -43,16 +43,13 @@
 
 object GearpumpMaterializer {
 
-  final val Debug = true
-
   final case class Edge(from: OutPort, to: InPort)
 
   final case class MaterializedValueSourceAttribute(mat: MaterializedValueNode) extends Attribute
 
   implicit def boolToAtomic(bool: Boolean): AtomicBoolean = new AtomicBoolean(bool)
 
-  def apply(strategy: Strategy)(implicit context: ActorRefFactory):
-  ExtendedActorMaterializer = {
+  def apply(strategy: Strategy)(implicit context: ActorRefFactory): ExtendedActorMaterializer = {
     val system = actorSystemOf(context)
 
     apply(ActorMaterializerSettings(
@@ -95,7 +92,7 @@
       case _ =>
         throw new IllegalArgumentException(
           s"""
-             |  context must be a ActorSystem or ActorContext, got [${context.getClass.getName}]
+            |  context must be a ActorSystem or ActorContext, got [${context.getClass.getName}]
           """.stripMargin
         )
     }
@@ -166,19 +163,10 @@
     system.scheduler.scheduleOnce(delay, task)(executionContext)
 
   override def materialize[Mat](runnableGraph: Graph[ClosedShape, Mat]): Mat = {
-    val initialAttributes = Attributes(
-      Attributes.InputBuffer(
-        settings.initialInputBufferSize,
-        settings.maxInputBufferSize
-      ) ::
-      ActorAttributes.Dispatcher(settings.dispatcher) ::
-      ActorAttributes.SupervisionStrategy(settings.supervisionDecider) ::
-      Nil)
-
     val info = Fusing.aggressive(runnableGraph).module.info
     val graph = GGraph.empty[Module, Edge]
 
-    info.allModules.foreach(module => {
+    info.subModules.foreach(module => {
       if (module.isCopied) {
         val original = module.asInstanceOf[CopiedModule].copyOf
         graph.addVertex(original)
@@ -201,9 +189,7 @@
       }
     })
 
-    if(Debug) {
-      printGraph(graph)
-    }
+    printGraph(graph)
 
     val subGraphs = GraphPartitioner(strategy).partition(graph)
     val matValues = subGraphs.foldLeft(mutable.Map.empty[Module, Any]) { (map, subGraph) =>
@@ -226,7 +212,7 @@
       }
     }).toList
     val matModule = subGraphs.last.graph.topologicalOrderIterator.toList.last
-    val mat2 = resolveMaterialized(matModule.materializedValueComputation, matValues)
+    resolveMaterialized(matModule.materializedValueComputation, matValues)
     val rt = Some(mat).flatMap(any => {
       any match {
         case promise: Promise[_] =>
@@ -235,7 +221,7 @@
           Some(other)
       }
     })
-    rt.getOrElse(null).asInstanceOf[Mat]
+    rt.orNull.asInstanceOf[Mat]
   }
 
   private def printGraph(graph: GGraph[Module, Edge]): Unit = {
@@ -269,10 +255,24 @@
   }
 
   override def materialize[Mat](runnableGraph: Graph[ClosedShape, Mat],
-      subflowFuser: GraphInterpreterShell => ActorRef): Mat = {
+      initialAttributes: Attributes): Mat = {
     materialize(runnableGraph)
   }
 
+  override def materialize[Mat](runnableGraph: Graph[ClosedShape, Mat],
+      subflowFuser: (GraphInterpreterShell) => ActorRef): Mat = {
+    materialize(runnableGraph)
+  }
+
+  override def materialize[Mat](runnableGraph: Graph[ClosedShape, Mat],
+      subflowFuser: (GraphInterpreterShell) => ActorRef, initialAttributes: Attributes): Mat = {
+    materialize(runnableGraph)
+  }
+
+  override def makeLogger(logSource: Class[_]): LoggingAdapter = {
+    logger
+  }
+
   def shutdown: Unit = {
     subMaterializers.values.foreach(_.shutdown)
   }
@@ -288,5 +288,8 @@
     case Ignore =>
       ()
   }
+
+
+
 }
 
diff --git a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/GearpumpMaterializerSession.scala b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/GearpumpMaterializerSession.scala
index 871dcf8..8a869d2 100644
--- a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/GearpumpMaterializerSession.scala
+++ b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/GearpumpMaterializerSession.scala
@@ -7,7 +7,7 @@
  * "License"); you may not use this 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,
diff --git a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test.scala b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test.scala
index 40cd556..52a45d9 100644
--- a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test.scala
+++ b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test.scala
@@ -7,7 +7,7 @@
  * "License"); you may not use this 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,
diff --git a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test10.scala b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test10.scala
index 71678c3..826cdcf 100644
--- a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test10.scala
+++ b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test10.scala
@@ -7,7 +7,7 @@
  * "License"); you may not use this 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,
diff --git a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test11.scala b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test11.scala
index b80398c..087c57d 100644
--- a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test11.scala
+++ b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test11.scala
@@ -7,7 +7,7 @@
  * "License"); you may not use this 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,
@@ -38,8 +38,8 @@
 
     implicit val system = ActorSystem("Test11", akkaConfig)
     implicit val materializer = GearpumpMaterializer()
-//    implicit val materializer =
-//    ActorMaterializer(ActorMaterializerSettings(system).withAutoFusing(false))
+    // implicit val materializer =
+    //   ActorMaterializer(ActorMaterializerSettings(system).withAutoFusing(false))
     implicit val ec = system.dispatcher
 
     val g = RunnableGraph.fromGraph(GraphDSL.create() {
diff --git a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test12.scala b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test12.scala
index a9e8b08..b4f4bce 100644
--- a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test12.scala
+++ b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test12.scala
@@ -7,7 +7,7 @@
  * "License"); you may not use this 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,
@@ -37,9 +37,9 @@
     import scala.concurrent.duration._
 
     implicit val system = ActorSystem("Test12", akkaConfig)
-//    implicit val materializer = ActorMaterializer(
-//      ActorMaterializerSettings(system).withAutoFusing(false)
-//    )
+    // implicit val materializer = ActorMaterializer(
+    //   ActorMaterializerSettings(system).withAutoFusing(false)
+    //   )
     implicit val materializer = GearpumpMaterializer()
     implicit val ec = system.dispatcher
 
diff --git a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test13.scala b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test13.scala
index 984c861..2e036cb 100644
--- a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test13.scala
+++ b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test13.scala
@@ -7,7 +7,7 @@
  * "License"); you may not use this 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,
diff --git a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test14.scala b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test14.scala
index 0542f43..c436130 100644
--- a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test14.scala
+++ b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test14.scala
@@ -7,7 +7,7 @@
  * "License"); you may not use this 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,
diff --git a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test15.scala b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test15.scala
index c2f8d5f..f4e4dbd 100644
--- a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test15.scala
+++ b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test15.scala
@@ -7,7 +7,7 @@
  * "License"); you may not use this 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,
diff --git a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test16.scala b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test16.scala
index eb0b5c7..9691496 100644
--- a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test16.scala
+++ b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test16.scala
@@ -7,7 +7,7 @@
  * "License"); you may not use this 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,
@@ -18,12 +18,11 @@
 
 package org.apache.gearpump.akkastream.example
 
-import akka.actor.{Actor, ActorSystem, Props}
+import akka.actor.ActorSystem
 import org.apache.gearpump.akkastream.GearpumpMaterializer
 import org.apache.gearpump.akkastream.scaladsl.{GearSink, GearSource}
-import akka.stream.scaladsl.Sink
 import org.apache.gearpump.cluster.main.ArgumentsParser
-import org.apache.gearpump.streaming.dsl.{CollectionDataSource, LoggerSink}
+import org.apache.gearpump.streaming.dsl.scalaapi.{CollectionDataSource, LoggerSink}
 import org.apache.gearpump.util.AkkaApp
 
 import scala.concurrent.Await
diff --git a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test2.scala b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test2.scala
index 21f1b8c..a6049cd 100644
--- a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test2.scala
+++ b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test2.scala
@@ -7,7 +7,7 @@
  * "License"); you may not use this 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,
diff --git a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test3.scala b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test3.scala
index 0a51078..24faeb3 100644
--- a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test3.scala
+++ b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test3.scala
@@ -7,7 +7,7 @@
  * "License"); you may not use this 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,
@@ -24,7 +24,7 @@
 import org.apache.gearpump.akkastream.scaladsl.GearSource
 import akka.stream.scaladsl.Sink
 import org.apache.gearpump.cluster.main.{ArgumentsParser, CLIOption}
-import org.apache.gearpump.streaming.dsl.CollectionDataSource
+import org.apache.gearpump.streaming.dsl.scalaapi.CollectionDataSource
 import org.apache.gearpump.util.AkkaApp
 
 import scala.concurrent.Await
diff --git a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test4.scala b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test4.scala
index 3cb69ce..6a44a35 100644
--- a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test4.scala
+++ b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test4.scala
@@ -7,7 +7,7 @@
  * "License"); you may not use this 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,
@@ -23,7 +23,7 @@
 import org.apache.gearpump.akkastream.GearpumpMaterializer
 import org.apache.gearpump.akkastream.scaladsl.GearSink
 import org.apache.gearpump.cluster.main.ArgumentsParser
-import org.apache.gearpump.streaming.dsl.LoggerSink
+import org.apache.gearpump.streaming.dsl.scalaapi.LoggerSink
 import org.apache.gearpump.util.AkkaApp
 
 import scala.concurrent.Await
diff --git a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test5.scala b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test5.scala
index 72e21c7..ad87a97 100644
--- a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test5.scala
+++ b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test5.scala
@@ -7,7 +7,7 @@
  * "License"); you may not use this 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,
diff --git a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test6.scala b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test6.scala
index 6f54933..a525471 100644
--- a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test6.scala
+++ b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test6.scala
@@ -7,7 +7,7 @@
  * "License"); you may not use this 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,
@@ -24,7 +24,7 @@
 import org.apache.gearpump.akkastream.GearpumpMaterializer
 import org.apache.gearpump.akkastream.scaladsl.GearSource
 import org.apache.gearpump.cluster.main.{ArgumentsParser, CLIOption}
-import org.apache.gearpump.streaming.dsl.CollectionDataSource
+import org.apache.gearpump.streaming.dsl.scalaapi.CollectionDataSource
 import org.apache.gearpump.util.AkkaApp
 
 import scala.concurrent.Await
diff --git a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test7.scala b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test7.scala
index be91610..8c837af 100644
--- a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test7.scala
+++ b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test7.scala
@@ -7,7 +7,7 @@
  * "License"); you may not use this 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,
diff --git a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test8.scala b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test8.scala
index 434aa33..ad2ac61 100644
--- a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test8.scala
+++ b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test8.scala
@@ -7,7 +7,7 @@
  * "License"); you may not use this 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,
diff --git a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test9.scala b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test9.scala
index 63f9e2d..66414e0 100644
--- a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test9.scala
+++ b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test9.scala
@@ -7,7 +7,7 @@
  * "License"); you may not use this 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,
diff --git a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/WikipediaApp.scala b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/WikipediaApp.scala
index 830f278..2a1e7ff 100644
--- a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/WikipediaApp.scala
+++ b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/WikipediaApp.scala
@@ -7,7 +7,7 @@
  * "License"); you may not use this 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,
diff --git a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/graph/GraphPartitioner.scala b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/graph/GraphPartitioner.scala
index c1e95bb..f7919c0 100644
--- a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/graph/GraphPartitioner.scala
+++ b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/graph/GraphPartitioner.scala
@@ -7,7 +7,7 @@
  * "License"); you may not use this 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,
diff --git a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/graph/LocalGraph.scala b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/graph/LocalGraph.scala
index c03fce2..fe86951 100644
--- a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/graph/LocalGraph.scala
+++ b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/graph/LocalGraph.scala
@@ -7,7 +7,7 @@
  * "License"); you may not use this 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,
diff --git a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/graph/RemoteGraph.scala b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/graph/RemoteGraph.scala
index 8fbe785..99ebe17 100644
--- a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/graph/RemoteGraph.scala
+++ b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/graph/RemoteGraph.scala
@@ -7,7 +7,7 @@
  * "License"); you may not use this 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,
@@ -79,7 +79,7 @@
       val materializer = new RemoteMaterializerImpl(graph, system)
       val (app, matValues) = materializer.materialize
 
-      val appId = context.submit(app)
+      val appId = context.submit(app).appId
       // scalastyle:off println
       println("sleep 5 second until the application is ready on cluster")
       // scalastyle:on println
diff --git a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/graph/SubGraph.scala b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/graph/SubGraph.scala
index a0395de..a74143e 100644
--- a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/graph/SubGraph.scala
+++ b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/graph/SubGraph.scala
@@ -7,7 +7,7 @@
  * "License"); you may not use this 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,
diff --git a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/materializer/LocalMaterializerImpl.scala b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/materializer/LocalMaterializerImpl.scala
index cbafcf5..477f4d3 100644
--- a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/materializer/LocalMaterializerImpl.scala
+++ b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/materializer/LocalMaterializerImpl.scala
@@ -7,7 +7,7 @@
  * "License"); you may not use this 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,
@@ -21,18 +21,16 @@
 import java.util.concurrent.atomic.AtomicBoolean
 import java.{util => ju}
 
-import _root_.org.apache.gearpump.util.{Graph => GGraph}
-import akka.NotUsed
+import org.apache.gearpump.util.{Graph => GGraph}
 import akka.actor.{ActorRef, ActorSystem, Cancellable, Deploy, PoisonPill}
 import akka.dispatch.Dispatchers
 import akka.event.{Logging, LoggingAdapter}
 import akka.stream.impl.StreamLayout._
 import akka.stream.impl._
 import akka.stream.impl.fusing.GraphInterpreter.GraphAssembly
+import akka.stream.impl.fusing.{ActorGraphInterpreter, Fold, GraphInterpreterShell, GraphModule, GraphStageModule}
 import akka.stream.impl.fusing.GraphStages.MaterializedValueSource
-import akka.stream.impl.fusing.{Map => _, _}
-import akka.stream.impl.io.{TLSActor, TlsModule}
-import akka.stream.scaladsl.{GraphDSL, Keep, ModuleExtractor, RunnableGraph}
+import akka.stream.scaladsl.ModuleExtractor
 import akka.stream.{ClosedShape, Graph => AkkaGraph, _}
 import org.apache.gearpump.akkastream.GearpumpMaterializer.Edge
 import org.apache.gearpump.akkastream.module.ReduceModule
@@ -121,25 +119,27 @@
           assignPort(stage.inPort, processor)
           assignPort(stage.outPort, processor.asInstanceOf[Publisher[Any]])
           matVal.put(atomic, mat)
-        case tls: TlsModule => // TODO solve this so TlsModule doesn't need special treatment here
-          val es = effectiveSettings(effectiveAttributes)
-          val props =
-            TLSActor.props(es, tls.sslContext, tls.sslConfig,
-              tls.firstSession, tls.role, tls.closing, tls.hostInfo)
-          val impl = actorOf(props, stageName(effectiveAttributes), es.dispatcher)
-          def factory(id: Int) = new ActorPublisher[Any](impl) {
-            override val wakeUpMsg = FanOut.SubstreamSubscribePending(id)
-          }
-          val publishers = Vector.tabulate(2)(factory)
-          impl ! FanOut.ExposedPublishers(publishers)
-
-          assignPort(tls.plainOut, publishers(TLSActor.UserOut))
-          assignPort(tls.cipherOut, publishers(TLSActor.TransportOut))
-
-          assignPort(tls.plainIn, FanIn.SubInput[Any](impl, TLSActor.UserIn))
-          assignPort(tls.cipherIn, FanIn.SubInput[Any](impl, TLSActor.TransportIn))
-
-          matVal.put(atomic, NotUsed)
+        // FIXME
+        //        case tls: TlsModule =>
+        // TODO solve this so TlsModule doesn't need special treatment here
+        //          val es = effectiveSettings(effectiveAttributes)
+        //          val props =
+        //            TLSActor.props(es, tls.sslContext, tls.sslConfig,
+        //              tls.firstSession, tls.role, tls.closing, tls.hostInfo)
+        //          val impl = actorOf(props, stageName(effectiveAttributes), es.dispatcher)
+        //          def factory(id: Int) = new ActorPublisher[Any](impl) {
+        //            override val wakeUpMsg = FanOut.SubstreamSubscribePending(id)
+        //          }
+        //          val publishers = Vector.tabulate(2)(factory)
+        //          impl ! FanOut.ExposedPublishers(publishers)
+        //
+        //          assignPort(tls.plainOut, publishers(TLSActor.UserOut))
+        //          assignPort(tls.cipherOut, publishers(TLSActor.TransportOut))
+        //
+        //          assignPort(tls.plainIn, FanIn.SubInput[Any](impl, TLSActor.UserIn))
+        //          assignPort(tls.cipherIn, FanIn.SubInput[Any](impl, TLSActor.TransportIn))
+        //
+        //          matVal.put(atomic, NotUsed)
         case graph: GraphModule =>
           matGraph(graph, effectiveAttributes, matVal)
         case stage: GraphStageModule =>
@@ -187,6 +187,11 @@
   }
 
   override def materialize[Mat](runnableGraph: AkkaGraph[ClosedShape, Mat],
+      initialAttributes: Attributes): Mat = {
+    materialize(runnableGraph)
+  }
+
+  override def materialize[Mat](runnableGraph: AkkaGraph[ClosedShape, Mat],
       subflowFuser: GraphInterpreterShell => ActorRef): Mat = {
 
     LocalMaterializerSession(ModuleExtractor.unapply(runnableGraph).get,
@@ -194,6 +199,15 @@
 
   }
 
+  override def materialize[Mat](runnableGraph: AkkaGraph[ClosedShape, Mat],
+      subflowFuser: (GraphInterpreterShell) => ActorRef, initialAttributes: Attributes): Mat = {
+    materialize(runnableGraph)
+  }
+
+  override def makeLogger(logSource: Class[_]): LoggingAdapter = {
+    logger
+  }
+
   def buildToplevelModule(graph: GGraph[Module, Edge]): Module = {
     var moduleInProgress: Module = EmptyModule
     graph.vertices.foreach(module => {
diff --git a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/materializer/RemoteMaterializerImpl.scala b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/materializer/RemoteMaterializerImpl.scala
index 936ac29..e065c90 100644
--- a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/materializer/RemoteMaterializerImpl.scala
+++ b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/materializer/RemoteMaterializerImpl.scala
@@ -7,7 +7,7 @@
  * "License"); you may not use this 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,
@@ -20,22 +20,24 @@
 
 import akka.actor.ActorSystem
 import akka.stream.impl.StreamLayout.Module
-import akka.stream.impl.Timers._
+import akka.stream.impl.Timers.{Completion, DelayInitial, Idle, IdleInject, IdleTimeoutBidi, Initial}
+import akka.stream.impl.fusing.{Batch, Collect, Delay, Drop, DropWhile, DropWithin, Filter, FlattenMerge, Fold, GraphStageModule, GroupBy, GroupedWithin, Intersperse, LimitWeighted, Log, MapAsync, MapAsyncUnordered, PrefixAndTail, Recover, Reduce, Scan, Split, StatefulMapConcat, SubSink, SubSource, Take, TakeWhile, TakeWithin, Map => FMap}
 import akka.stream.impl.fusing.GraphStages.{MaterializedValueSource, SimpleLinearGraphStage, SingleSource, TickSource}
-import akka.stream.impl.fusing.{Map => FMap, _}
 import akka.stream.impl.io.IncomingConnectionStage
-import akka.stream.impl.{HeadOptionStage, Stages, Throttle}
-import akka.stream.scaladsl._
+import akka.stream.impl.{HeadOptionStage, Stages, Throttle, Unfold, UnfoldAsync}
+import akka.stream.scaladsl.{Balance, Broadcast, Concat, Interleave, Merge, MergePreferred, MergeSorted, ModuleExtractor, Unzip, Zip, ZipWith2}
 import akka.stream.stage.AbstractStage.PushPullGraphStageWithMaterializedValue
 import akka.stream.stage.GraphStage
 import org.apache.gearpump.akkastream.GearAttributes
 import org.apache.gearpump.akkastream.GearpumpMaterializer.Edge
-import org.apache.gearpump.akkastream.module._
-import org.apache.gearpump.akkastream.task._
+import org.apache.gearpump.akkastream.module.{GroupByModule, ProcessorModule, ReduceModule, SinkBridgeModule, SinkTaskModule, SourceBridgeModule, SourceTaskModule}
+import org.apache.gearpump.akkastream.task.{BalanceTask, BatchTask, BroadcastTask, ConcatTask, DelayInitialTask, DropWithinTask, FlattenMergeTask, FoldTask, GraphTask, GroupedWithinTask, InterleaveTask, MapAsyncTask, MergeTask, SingleSourceTask, SinkBridgeTask, SourceBridgeTask, StatefulMapConcatTask, TakeWithinTask, ThrottleTask, TickSourceTask, Zip2Task}
+import org.apache.gearpump.akkastream.task.TickSourceTask.{INITIAL_DELAY, INTERVAL, TICK}
 import org.apache.gearpump.cluster.UserConfig
-import org.apache.gearpump.streaming.dsl.StreamApp
-import org.apache.gearpump.streaming.dsl.plan._
-import org.apache.gearpump.streaming.dsl.plan.functions.FlatMapFunction
+import org.apache.gearpump.streaming.dsl.plan.functions.FlatMapper
+import org.apache.gearpump.streaming.dsl.plan.{ChainableOp, DataSinkOp, DataSourceOp, Direct, GroupByOp, MergeOp, Op, OpEdge, ProcessorOp, Shuffle}
+import org.apache.gearpump.streaming.dsl.scalaapi.StreamApp
+import org.apache.gearpump.streaming.dsl.scalaapi.functions.FlatMapFunction
 import org.apache.gearpump.streaming.dsl.window.api.CountWindow
 import org.apache.gearpump.streaming.dsl.window.impl.GroupAlsoByWindow
 import org.apache.gearpump.streaming.{ProcessorId, StreamApplication}
@@ -162,7 +164,8 @@
         case sinkBridge: SinkBridgeModule[_, _] =>
           ProcessorOp(classOf[SinkBridgeTask], parallelism, conf, "sink")
         case groupBy: GroupByModule[_, _] =>
-          GroupByOp(groupBy.groupBy, parallelism, "groupBy", conf)
+          GroupByOp(GroupAlsoByWindow(groupBy.groupBy, CountWindow.apply(1).accumulating),
+            parallelism, "groupBy", conf)
         case reduce: ReduceModule[_] =>
           reduceOp(reduce.f, conf)
         case graphStage: GraphStageModule =>
@@ -192,7 +195,6 @@
       parallelism: Int, conf: UserConfig): Op = {
     module.stage match {
       case tickSource: TickSource[_] =>
-        import TickSourceTask._
         val tick: AnyRef = tickSource.tick.asInstanceOf[AnyRef]
         val tiConf = conf.withValue[FiniteDuration](INITIAL_DELAY, tickSource.initialDelay).
           withValue[FiniteDuration](INTERVAL, tickSource.interval).
@@ -322,9 +324,9 @@
         // TODO
         null
       case unzip: Unzip[_, _] =>
-//        ProcessorOp(classOf[Unzip2Task[_, _, _]], parallelism,
-//          conf.withValue(
-//            Unzip2Task.UNZIP2_FUNCTION, Unzip2Task.UnZipFunction(unzip.unzipper)), "unzip")
+        // ProcessorOp(classOf[Unzip2Task[_, _, _]], parallelism,
+        //   conf.withValue(
+        //     Unzip2Task.UNZIP2_FUNCTION, Unzip2Task.UnZipFunction(unzip.unzipper)), "unzip")
         // TODO
         null
       case zip: Zip[_, _] =>
@@ -388,9 +390,9 @@
   private def translateSymbolic(stage: PushPullGraphStageWithMaterializedValue[_, _, _, _],
       conf: UserConfig): Op = {
     stage match {
-      case symbolicGraphStage: Stages.SymbolicGraphStage[_, _, _] =>
-        symbolicGraphStage.symbolicStage match {
-          case buffer: Stages.Buffer[_] =>
+      case symbolicGraphStage: Stages.SymbolicGraphStage[_, _, _]
+        if symbolicGraphStage.symbolicStage.attributes.equals(
+          Stages.DefaultAttributes.buffer) => {
             // ignore the buffering operation
             identity("buffer", conf)
         }
@@ -478,7 +480,7 @@
 
   def flatMapOp[In, Out](fun: In => TraversableOnce[Out], description: String,
       conf: UserConfig): Op = {
-    ChainableOp(new FlatMapFunction[In, Out](fun, description), conf)
+    ChainableOp(new FlatMapper(FlatMapFunction[In, Out](fun), description), conf)
   }
 
   def conflateOp[In, Out](seed: In => Out, aggregate: (Out, In) => Out,
diff --git a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/module/BridgeModule.scala b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/module/BridgeModule.scala
index 35d0e88..5b8c71b 100644
--- a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/module/BridgeModule.scala
+++ b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/module/BridgeModule.scala
@@ -7,7 +7,7 @@
  * "License"); you may not use this 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,
diff --git a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/module/DummyModule.scala b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/module/DummyModule.scala
index 2c430d5..ea76bb0 100644
--- a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/module/DummyModule.scala
+++ b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/module/DummyModule.scala
@@ -7,7 +7,7 @@
  * "License"); you may not use this 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,
diff --git a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/module/GearpumpTaskModule.scala b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/module/GearpumpTaskModule.scala
index 7555244..dfbbee9 100644
--- a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/module/GearpumpTaskModule.scala
+++ b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/module/GearpumpTaskModule.scala
@@ -7,7 +7,7 @@
  * "License"); you may not use this 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,
diff --git a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/module/GroupByModule.scala b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/module/GroupByModule.scala
index 4465886..b06dd0e 100644
--- a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/module/GroupByModule.scala
+++ b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/module/GroupByModule.scala
@@ -7,7 +7,7 @@
  * "License"); you may not use this 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,
diff --git a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/module/ReduceModule.scala b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/module/ReduceModule.scala
index 295556f..462d967 100644
--- a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/module/ReduceModule.scala
+++ b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/module/ReduceModule.scala
@@ -7,7 +7,7 @@
  * "License"); you may not use this 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,
diff --git a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/scaladsl/Api.scala b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/scaladsl/Api.scala
index 80619ef..8e43c16 100644
--- a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/scaladsl/Api.scala
+++ b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/scaladsl/Api.scala
@@ -7,7 +7,7 @@
  * "License"); you may not use this 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,
diff --git a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/BalanceTask.scala b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/BalanceTask.scala
index 5139117..43f07c4 100644
--- a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/BalanceTask.scala
+++ b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/BalanceTask.scala
@@ -7,7 +7,7 @@
  * "License"); you may not use this 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,
diff --git a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/BatchTask.scala b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/BatchTask.scala
index 582327b..5c2485b 100644
--- a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/BatchTask.scala
+++ b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/BatchTask.scala
@@ -7,7 +7,7 @@
  * "License"); you may not use this 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,
diff --git a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/BroadcastTask.scala b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/BroadcastTask.scala
index 9f1194f..292468d 100644
--- a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/BroadcastTask.scala
+++ b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/BroadcastTask.scala
@@ -7,7 +7,7 @@
  * "License"); you may not use this 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,
diff --git a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/ConcatTask.scala b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/ConcatTask.scala
index 241fa76..b77b9bd 100644
--- a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/ConcatTask.scala
+++ b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/ConcatTask.scala
@@ -7,7 +7,7 @@
  * "License"); you may not use this 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,
diff --git a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/DelayInitialTask.scala b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/DelayInitialTask.scala
index d6c347a..7c335dc 100644
--- a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/DelayInitialTask.scala
+++ b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/DelayInitialTask.scala
@@ -7,7 +7,7 @@
  * "License"); you may not use this 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,
diff --git a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/DropWithinTask.scala b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/DropWithinTask.scala
index 9da26b1..0c54829 100644
--- a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/DropWithinTask.scala
+++ b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/DropWithinTask.scala
@@ -7,7 +7,7 @@
  * "License"); you may not use this 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,
diff --git a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/FlattenMergeTask.scala b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/FlattenMergeTask.scala
index 512164d..14ff537 100644
--- a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/FlattenMergeTask.scala
+++ b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/FlattenMergeTask.scala
@@ -7,7 +7,7 @@
  * "License"); you may not use this 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,
diff --git a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/FoldTask.scala b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/FoldTask.scala
index e2f02d8..d982ebd 100644
--- a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/FoldTask.scala
+++ b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/FoldTask.scala
@@ -7,7 +7,7 @@
  * "License"); you may not use this 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,
diff --git a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/GraphTask.scala b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/GraphTask.scala
index 8e7a2df..3310ab9 100644
--- a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/GraphTask.scala
+++ b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/GraphTask.scala
@@ -7,7 +7,7 @@
  * "License"); you may not use this 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,
diff --git a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/GroupedWithinTask.scala b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/GroupedWithinTask.scala
index 29d9c91..eaf2b3f 100644
--- a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/GroupedWithinTask.scala
+++ b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/GroupedWithinTask.scala
@@ -7,7 +7,7 @@
  * "License"); you may not use this 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,
diff --git a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/InterleaveTask.scala b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/InterleaveTask.scala
index 837de6b..741ec43 100644
--- a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/InterleaveTask.scala
+++ b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/InterleaveTask.scala
@@ -7,7 +7,7 @@
  * "License"); you may not use this 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,
diff --git a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/MapAsyncTask.scala b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/MapAsyncTask.scala
index 387116d..daa1afc 100644
--- a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/MapAsyncTask.scala
+++ b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/MapAsyncTask.scala
@@ -7,7 +7,7 @@
  * "License"); you may not use this 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,
diff --git a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/MergeTask.scala b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/MergeTask.scala
index 2b1cd33..ad18f72 100644
--- a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/MergeTask.scala
+++ b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/MergeTask.scala
@@ -7,12 +7,12 @@
  * "License"); you may not use this 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.
- * SeG the License for the specific language governing permissions and
+ * See the License for the specific language governing permissions and
  * limitations under the License.
  */
 
diff --git a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/SingleSourceTask.scala b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/SingleSourceTask.scala
index 1ff9ccd..458bb4e 100644
--- a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/SingleSourceTask.scala
+++ b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/SingleSourceTask.scala
@@ -7,7 +7,7 @@
  * "License"); you may not use this 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,
diff --git a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/SinkBridgeTask.scala b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/SinkBridgeTask.scala
index 05011e9..1b9c4e3 100644
--- a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/SinkBridgeTask.scala
+++ b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/SinkBridgeTask.scala
@@ -7,7 +7,7 @@
  * "License"); you may not use this 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,
diff --git a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/SourceBridgeTask.scala b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/SourceBridgeTask.scala
index b0eda19..054b483 100644
--- a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/SourceBridgeTask.scala
+++ b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/SourceBridgeTask.scala
@@ -7,7 +7,7 @@
  * "License"); you may not use this 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,
diff --git a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/StatefulMapConcatTask.scala b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/StatefulMapConcatTask.scala
index bf2c14f..a0674bc 100644
--- a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/StatefulMapConcatTask.scala
+++ b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/StatefulMapConcatTask.scala
@@ -7,7 +7,7 @@
  * "License"); you may not use this 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,
diff --git a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/TakeWithinTask.scala b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/TakeWithinTask.scala
index ef43fbe..9559d8f 100644
--- a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/TakeWithinTask.scala
+++ b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/TakeWithinTask.scala
@@ -7,7 +7,7 @@
  * "License"); you may not use this 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,
diff --git a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/ThrottleTask.scala b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/ThrottleTask.scala
index 4e09bf2..3c7ad87 100644
--- a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/ThrottleTask.scala
+++ b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/ThrottleTask.scala
@@ -7,7 +7,7 @@
  * "License"); you may not use this 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,
diff --git a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/TickSourceTask.scala b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/TickSourceTask.scala
index b3850ca..d99d2db 100644
--- a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/TickSourceTask.scala
+++ b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/TickSourceTask.scala
@@ -7,7 +7,7 @@
  * "License"); you may not use this 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,
diff --git a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/Unzip2Task.scala b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/Unzip2Task.scala
index 7dd91fc..005d018 100644
--- a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/Unzip2Task.scala
+++ b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/Unzip2Task.scala
@@ -7,7 +7,7 @@
  * "License"); you may not use this 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,
diff --git a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/Zip2Task.scala b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/Zip2Task.scala
index a35b133..7e0c082 100644
--- a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/Zip2Task.scala
+++ b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/Zip2Task.scala
@@ -7,7 +7,7 @@
  * "License"); you may not use this 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,
diff --git a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/util/MaterializedValueOps.scala b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/util/MaterializedValueOps.scala
index c9fe67d..6ad90df 100644
--- a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/util/MaterializedValueOps.scala
+++ b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/util/MaterializedValueOps.scala
@@ -7,7 +7,7 @@
  * "License"); you may not use this 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,
diff --git a/experiments/cgroup/src/main/java/org/apache/gearpump/cluster/utils/SystemOperation.java b/experiments/cgroup/src/main/java/org/apache/gearpump/cluster/utils/SystemOperation.java
index 5b2a890..f52afc7 100644
--- a/experiments/cgroup/src/main/java/org/apache/gearpump/cluster/utils/SystemOperation.java
+++ b/experiments/cgroup/src/main/java/org/apache/gearpump/cluster/utils/SystemOperation.java
@@ -17,11 +17,12 @@
  */
 package org.apache.gearpump.cluster.utils;
 
-import org.apache.commons.io.IOUtils;
+import com.google.common.io.CharStreams;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import java.io.IOException;
+import java.io.InputStreamReader;
 
 public class SystemOperation {
 
@@ -44,8 +45,8 @@
     Process process = new ProcessBuilder(new String[]{"/bin/bash", "-c", cmd}).start();
     try {
       process.waitFor();
-      String output = IOUtils.toString(process.getInputStream());
-      String errorOutput = IOUtils.toString(process.getErrorStream());
+      String output = CharStreams.toString(new InputStreamReader(process.getInputStream()));
+      String errorOutput = CharStreams.toString(new InputStreamReader(process.getErrorStream()));
       LOG.debug("Shell Output: " + output);
       if (errorOutput.length() != 0) {
         LOG.error("Shell Error Output: " + errorOutput);
diff --git a/experiments/cgroup/src/main/scala/org/apache/gearpump/cluster/worker/CGroupProcessLauncher.scala b/experiments/cgroup/src/main/scala/org/apache/gearpump/cluster/worker/CGroupProcessLauncher.scala
index 74a4047..bf291cf 100644
--- a/experiments/cgroup/src/main/scala/org/apache/gearpump/cluster/worker/CGroupProcessLauncher.scala
+++ b/experiments/cgroup/src/main/scala/org/apache/gearpump/cluster/worker/CGroupProcessLauncher.scala
@@ -43,7 +43,7 @@
   }
 
   override def createProcess(
-    appId: Int, executorId: Int, resource: Resource, appConfig: Config, options: Array[String],
+      appId: Int, executorId: Int, resource: Resource, appConfig: Config, options: Array[String],
     classPath: Array[String], mainClass: String, arguments: Array[String]): RichProcess = {
     val cgroupCommand = if (executorId != APP_MASTER) {
       cgroupManager.map(_.startNewExecutor(appConfig, resource.slots, appId,
diff --git a/experiments/redis/src/main/scala/org/apache/gearpump/redis/RedisMessage.scala b/experiments/redis/src/main/scala/org/apache/gearpump/redis/RedisMessage.scala
index 84dec70..ea738d6 100644
--- a/experiments/redis/src/main/scala/org/apache/gearpump/redis/RedisMessage.scala
+++ b/experiments/redis/src/main/scala/org/apache/gearpump/redis/RedisMessage.scala
@@ -20,7 +20,6 @@
 import java.nio.charset.Charset
 
 object RedisMessage {
-
   private def toBytes(strings: List[String]): List[Array[Byte]] =
     strings.map(string => string.getBytes(Charset.forName("UTF8")))
 
@@ -48,11 +47,10 @@
      * @param latitude
      * @param member
      */
-    case class GEOADD(key: Array[Byte], longitude: Double,
-                      latitude: Double, member: Array[Byte]) {
-      def this(key: String, longitude: Double,
-               latitude: Double, member: String) =
+    case class GEOADD(key: Array[Byte], longitude: Double, latitude: Double, member: Array[Byte]) {
+      def this(key: String, longitude: Double, latitude: Double, member: String) = {
         this(toBytes(key), longitude, latitude, toBytes(member))
+      }
     }
 
   }
@@ -66,7 +64,9 @@
      * @param field
      */
     case class HDEL(key: Array[Byte], field: Array[Byte]) {
-      def this(key: String, field: String) = this(toBytes(key), toBytes(field))
+      def this(key: String, field: String) = {
+        this(toBytes(key), toBytes(field))
+      }
     }
 
     /**
@@ -77,8 +77,9 @@
      * @param increment
      */
     case class HINCRBY(key: Array[Byte], field: Array[Byte], increment: Long) {
-      def this(key: String, field: String, increment: Long) =
+      def this(key: String, field: String, increment: Long) = {
         this(toBytes(key), toBytes(field), increment)
+      }
     }
 
     /**
@@ -89,8 +90,9 @@
      * @param increment
      */
     case class HINCRBYFLOAT(key: Array[Byte], field: Array[Byte], increment: Float) {
-      def this(key: String, field: String, increment: Float) =
+      def this(key: String, field: String, increment: Float) = {
         this(toBytes(key), toBytes(field), increment)
+      }
     }
 
 
@@ -102,8 +104,9 @@
      * @param value
      */
     case class HSET(key: Array[Byte], field: Array[Byte], value: Array[Byte]) {
-      def this(key: String, field: String, value: String) =
+      def this(key: String, field: String, value: String) = {
         this(toBytes(key), toBytes(field), toBytes(value))
+      }
     }
 
     /**
@@ -114,8 +117,9 @@
      * @param value
      */
     case class HSETNX(key: Array[Byte], field: Array[Byte], value: Array[Byte]) {
-      def this(key: String, field: String, value: String) =
+      def this(key: String, field: String, value: String) = {
         this(toBytes(key), toBytes(field), toBytes(value))
+      }
     }
 
   }
@@ -142,8 +146,9 @@
      * @param value
      */
     case class LPUSH(key: Array[Byte], value: Array[Byte]) {
-
-      def this(key: String, value: String) = this(key, toBytes(value))
+      def this(key: String, value: String) = {
+        this(toBytes(key), toBytes(value))
+      }
     }
 
     /**
@@ -153,7 +158,9 @@
      * @param value
      */
     case class LPUSHX(key: Array[Byte], value: Array[Byte]) {
-      def this(key: String, value: String) = this(toBytes(key), toBytes(value))
+      def this(key: String, value: String) = {
+        this(toBytes(key), toBytes(value))
+      }
     }
 
     /**
@@ -164,7 +171,9 @@
      * @param value
      */
     case class LSET(key: Array[Byte], index: Long, value: Array[Byte]) {
-      def this(key: String, index: Long, value: String) = this(toBytes(key), index, toBytes(value))
+      def this(key: String, index: Long, value: String) = {
+        this(toBytes(key), index, toBytes(value))
+      }
     }
 
     /**
@@ -174,8 +183,9 @@
      * @param value
      */
     case class RPUSH(key: Array[Byte], value: Array[Byte]) {
-
-      def this(key: String, value: String) = this(key, toBytes(value))
+      def this(key: String, value: String) = {
+        this(toBytes(key), toBytes(value))
+      }
     }
 
     /**
@@ -185,7 +195,9 @@
      * @param value
      */
     case class RPUSHX(key: Array[Byte], value: Array[Byte]) {
-      def this(key: String, value: String) = this(toBytes(key), toBytes(value))
+      def this(key: String, value: String) = {
+        this(toBytes(key), toBytes(value))
+      }
     }
 
   }
@@ -198,8 +210,9 @@
      * @param message
      */
     case class DEL(message: Array[Byte]) {
-
-      def this(message: String) = this(toBytes(message))
+      def this(message: String) = {
+        this(toBytes(message))
+      }
     }
 
     /**
@@ -208,7 +221,9 @@
      * @param key
      */
     case class EXPIRE(key: Array[Byte], seconds: Int) {
-      def this(key: String, seconds: Int) = this(toBytes(key), seconds)
+      def this(key: String, seconds: Int) = {
+        this(toBytes(key), seconds)
+      }
     }
 
     /**
@@ -218,7 +233,9 @@
      * @param timestamp
      */
     case class EXPIREAT(key: Array[Byte], timestamp: Long) {
-      def this(key: String, timestamp: Long) = this(toBytes(key), timestamp)
+      def this(key: String, timestamp: Long) = {
+        this(toBytes(key), timestamp)
+      }
     }
 
     /**
@@ -230,9 +247,11 @@
      * @param database
      * @param timeout
      */
-    case class MIGRATE(host: Array[Byte], port: Int, key: Array[Byte], database: Int, timeout: Int) {
-      def this(host: String, port: Int, key: String, database: Int, timeout: Int) =
+    case class MIGRATE(host: Array[Byte], port: Int, key: Array[Byte],
+        database: Int, timeout: Int) {
+      def this(host: String, port: Int, key: String, database: Int, timeout: Int) = {
         this(toBytes(host), port, toBytes(key), database, timeout)
+      }
     }
 
     /**
@@ -242,7 +261,9 @@
      * @param db
      */
     case class MOVE(key: Array[Byte], db: Int) {
-      def this(key: String, db: Int) = this(toBytes(key), db)
+      def this(key: String, db: Int) = {
+        this(toBytes(key), db)
+      }
     }
 
     /**
@@ -251,7 +272,9 @@
      * @param key
      */
     case class PERSIST(key: Array[Byte]) {
-      def this(key: String) = this(toBytes(key))
+      def this(key: String) = {
+        this(toBytes(key))
+      }
     }
 
     /**
@@ -261,7 +284,9 @@
      * @param milliseconds
      */
     case class PEXPIRE(key: Array[Byte], milliseconds: Long) {
-      def this(key: String, milliseconds: Long) = this(toBytes(key), milliseconds)
+      def this(key: String, milliseconds: Long) = {
+        this(toBytes(key), milliseconds)
+      }
     }
 
     /**
@@ -271,7 +296,9 @@
      * @param timestamp
      */
     case class PEXPIREAT(key: Array[Byte], timestamp: Long) {
-      def this(key: String, milliseconds: Long) = this(toBytes(key), milliseconds)
+      def this(key: String, milliseconds: Long) = {
+        this(toBytes(key), milliseconds)
+      }
     }
 
     /**
@@ -281,7 +308,9 @@
      * @param newKey
      */
     case class RENAME(key: Array[Byte], newKey: Array[Byte]) {
-      def this(key: String, newKey: String) = this(toBytes(key), toBytes(newKey))
+      def this(key: String, newKey: String) = {
+        this(toBytes(key), toBytes(newKey))
+      }
     }
 
     /**
@@ -291,7 +320,9 @@
      * @param newKey
      */
     case class RENAMENX(key: Array[Byte], newKey: Array[Byte]) {
-      def this(key: String, newKey: String) = this(toBytes(key), toBytes(newKey))
+      def this(key: String, newKey: String) = {
+        this(toBytes(key), toBytes(newKey))
+      }
     }
 
   }
@@ -306,8 +337,9 @@
      * @param members
      */
     case class SADD(key: Array[Byte], members: Array[Byte]) {
-
-      def this(key: String, members: String) = this(key, toBytes(members))
+      def this(key: String, members: String) = {
+        this(toBytes(key), toBytes(members))
+      }
     }
 
 
@@ -319,8 +351,9 @@
      * @param member
      */
     case class SMOVE(source: Array[Byte], destination: Array[Byte], member: Array[Byte]) {
-      def this(source: String, destination: String, member: String) =
+      def this(source: String, destination: String, member: String) = {
         this(toBytes(source), toBytes(destination), toBytes(member))
+      }
     }
 
 
@@ -331,8 +364,9 @@
      * @param member
      */
     case class SREM(key: Array[Byte], member: Array[Byte]) {
-
-      def this(key: String, member: String) = this(key, toBytes(member))
+      def this(key: String, member: String) = {
+        this(toBytes(key), toBytes(member))
+      }
     }
 
   }
@@ -346,7 +380,9 @@
      * @param value
      */
     case class APPEND(key: Array[Byte], value: Array[Byte]) {
-      def this(key: String, value: String) = this(toBytes(key), toBytes(value))
+      def this(key: String, value: String) = {
+        this(toBytes(key), toBytes(value))
+      }
     }
 
     /**
@@ -355,7 +391,9 @@
      * @param key
      */
     case class DECR(key: Array[Byte]) {
-      def this(key: String) = this(toBytes(key))
+      def this(key: String) = {
+        this(toBytes(key))
+      }
     }
 
     /**
@@ -365,7 +403,9 @@
      * @param decrement
      */
     case class DECRBY(key: Array[Byte], decrement: Int) {
-      def this(key: String, decrement: Int) = this(toBytes(key), decrement)
+      def this(key: String, decrement: Int) = {
+        this(toBytes(key), decrement)
+      }
     }
 
     /**
@@ -374,7 +414,9 @@
      * @param key
      */
     case class INCR(key: Array[Byte]) {
-      def this(key: String) = this(toBytes(key))
+      def this(key: String) = {
+        this(toBytes(key))
+      }
     }
 
     /**
@@ -384,7 +426,9 @@
      * @param increment
      */
     case class INCRBY(key: Array[Byte], increment: Int) {
-      def this(key: String, increment: Int) = this(toBytes(key), increment)
+      def this(key: String, increment: Int) = {
+        this(toBytes(key), increment)
+      }
     }
 
     /**
@@ -394,7 +438,9 @@
      * @param increment
      */
     case class INCRBYFLOAT(key: Array[Byte], increment: Double) {
-      def this(key: String, increment: Number) = this(toBytes(key), increment)
+      def this(key: String, increment: Double) = {
+        this(toBytes(key), increment)
+      }
     }
 
 
@@ -405,7 +451,9 @@
      * @param value
      */
     case class SET(key: Array[Byte], value: Array[Byte]) {
-      def this(key: String, value: String) = this(toBytes(key), toBytes(value))
+      def this(key: String, value: String) = {
+        this(toBytes(key), toBytes(value))
+      }
     }
 
     /**
@@ -416,7 +464,9 @@
      * @param value
      */
     case class SETBIT(key: Array[Byte], offset: Long, value: Array[Byte]) {
-      def this(key: String, offset: Long, value: String) = this(toBytes(key), offset, toBytes(value))
+      def this(key: String, offset: Long, value: String) = {
+        this(toBytes(key), offset, toBytes(value))
+      }
     }
 
     /**
@@ -427,7 +477,9 @@
      * @param value
      */
     case class SETEX(key: Array[Byte], seconds: Int, value: Array[Byte]) {
-      def this(key: String, seconds: Int, value: String) = this(toBytes(key), seconds, toBytes(value))
+      def this(key: String, seconds: Int, value: String) = {
+        this(toBytes(key), seconds, toBytes(value))
+      }
     }
 
     /**
@@ -437,7 +489,9 @@
      * @param value
      */
     case class SETNX(key: Array[Byte], value: Array[Byte]) {
-      def this(key: String, value: String) = this(toBytes(key), toBytes(value))
+      def this(key: String, value: String) = {
+        this(toBytes(key), toBytes(value))
+      }
     }
 
     /**
@@ -448,9 +502,9 @@
      * @param value
      */
     case class SETRANGE(key: Array[Byte], offset: Int, value: Array[Byte]) {
-      def this(key: String, offset: Int, value: String) = this(toBytes(key), offset, toBytes(value))
+      def this(key: String, offset: Int, value: String) = {
+        this(toBytes(key), offset, toBytes(value))
+      }
     }
-
   }
-
 }
diff --git a/experiments/redis/src/main/scala/org/apache/gearpump/redis/RedisSink.scala b/experiments/redis/src/main/scala/org/apache/gearpump/redis/RedisSink.scala
index 3f75949..36a9fe3 100644
--- a/experiments/redis/src/main/scala/org/apache/gearpump/redis/RedisSink.scala
+++ b/experiments/redis/src/main/scala/org/apache/gearpump/redis/RedisSink.scala
@@ -32,20 +32,20 @@
 import redis.clients.jedis.Protocol.{DEFAULT_DATABASE, DEFAULT_HOST, DEFAULT_PORT, DEFAULT_TIMEOUT}
 
 /**
-  * Save message in Redis Instance
-  *
-  * @param host
-  * @param port
-  * @param timeout
-  * @param database
-  * @param password
-  */
+ * Save message in Redis Instance
+ *
+ * @param host
+ * @param port
+ * @param timeout
+ * @param database
+ * @param password
+ */
 class RedisSink(
-                    host: String = DEFAULT_HOST,
-                    port: Int = DEFAULT_PORT,
-                    timeout: Int = DEFAULT_TIMEOUT,
-                    database: Int = DEFAULT_DATABASE,
-                    password: String = "") extends DataSink {
+    host: String = DEFAULT_HOST,
+    port: Int = DEFAULT_PORT,
+    timeout: Int = DEFAULT_TIMEOUT,
+    database: Int = DEFAULT_DATABASE,
+    password: String = "") extends DataSink {
 
   private val LOG = LogUtil.getLogger(getClass)
   @transient private lazy val client = new Jedis(host, port, timeout)
@@ -59,7 +59,6 @@
   }
 
   override def write(message: Message): Unit = {
-
     message.msg match {
       // GEO
       case msg: GEOADD => client.geoadd(msg.key, msg.longitude, msg.latitude, msg.member)
diff --git a/experiments/storm/src/main/scala/org/apache/gearpump/experiments/storm/main/GearpumpNimbus.scala b/experiments/storm/src/main/scala/org/apache/gearpump/experiments/storm/main/GearpumpNimbus.scala
index 544a4eb..df1de06 100644
--- a/experiments/storm/src/main/scala/org/apache/gearpump/experiments/storm/main/GearpumpNimbus.scala
+++ b/experiments/storm/src/main/scala/org/apache/gearpump/experiments/storm/main/GearpumpNimbus.scala
@@ -163,7 +163,7 @@
       .withValue[JMap[AnyRef, AnyRef]](StormConstants.STORM_CONFIG, stormConfig)
     val app = StreamApplication(name, processorGraph, config)
     LOG.info(s"jar file uploaded to $uploadedJarLocation")
-    val appId = clientContext.submit(app, uploadedJarLocation, workerNum)
+    val appId = clientContext.submit(app, uploadedJarLocation, workerNum).appId
     applications += name -> appId
     topologies += name -> TopologyData(topology, stormConfig, uploadedJarLocation)
     LOG.info(s"Storm Application $appId submitted")
diff --git a/experiments/storm/src/main/scala/org/apache/gearpump/experiments/storm/partitioner/StormPartitioner.scala b/experiments/storm/src/main/scala/org/apache/gearpump/experiments/storm/partitioner/StormPartitioner.scala
index aaa0a99..4969314 100644
--- a/experiments/storm/src/main/scala/org/apache/gearpump/experiments/storm/partitioner/StormPartitioner.scala
+++ b/experiments/storm/src/main/scala/org/apache/gearpump/experiments/storm/partitioner/StormPartitioner.scala
@@ -20,7 +20,7 @@
 
 import org.apache.gearpump.Message
 import org.apache.gearpump.experiments.storm.topology.GearpumpTuple
-import org.apache.gearpump.partitioner.{MulticastPartitioner, Partitioner}
+import org.apache.gearpump.streaming.partitioner.{MulticastPartitioner, Partitioner}
 
 /**
  * Partitioner bound to a target Storm component
diff --git a/experiments/storm/src/main/scala/org/apache/gearpump/experiments/storm/util/GraphBuilder.scala b/experiments/storm/src/main/scala/org/apache/gearpump/experiments/storm/util/GraphBuilder.scala
index 777acab..e3f1339 100644
--- a/experiments/storm/src/main/scala/org/apache/gearpump/experiments/storm/util/GraphBuilder.scala
+++ b/experiments/storm/src/main/scala/org/apache/gearpump/experiments/storm/util/GraphBuilder.scala
@@ -20,7 +20,7 @@
 
 import org.apache.gearpump.experiments.storm.partitioner.StormPartitioner
 import org.apache.gearpump.experiments.storm.topology.GearpumpStormTopology
-import org.apache.gearpump.partitioner.Partitioner
+import org.apache.gearpump.streaming.partitioner.Partitioner
 import org.apache.gearpump.streaming.Processor
 import org.apache.gearpump.streaming.task.Task
 import org.apache.gearpump.util.Graph
diff --git a/experiments/storm/src/test/scala/org/apache/gearpump/experiments/storm/partitioner/StormPartitionerSpec.scala b/experiments/storm/src/test/scala/org/apache/gearpump/experiments/storm/partitioner/StormPartitionerSpec.scala
index 5513423..5fc631b 100644
--- a/experiments/storm/src/test/scala/org/apache/gearpump/experiments/storm/partitioner/StormPartitionerSpec.scala
+++ b/experiments/storm/src/test/scala/org/apache/gearpump/experiments/storm/partitioner/StormPartitionerSpec.scala
@@ -27,7 +27,7 @@
 
 import org.apache.gearpump.Message
 import org.apache.gearpump.experiments.storm.topology.GearpumpTuple
-import org.apache.gearpump.partitioner.Partitioner
+import org.apache.gearpump.streaming.partitioner.Partitioner
 
 class StormPartitionerSpec extends PropSpec with PropertyChecks with Matchers {
 
diff --git a/experiments/yarn/src/main/scala/org/apache/gearpump/experiments/yarn/Constants.scala b/experiments/yarn/src/main/scala/org/apache/gearpump/experiments/yarn/Constants.scala
index 33c3e97..95c95c7 100644
--- a/experiments/yarn/src/main/scala/org/apache/gearpump/experiments/yarn/Constants.scala
+++ b/experiments/yarn/src/main/scala/org/apache/gearpump/experiments/yarn/Constants.scala
@@ -19,6 +19,7 @@
 package org.apache.gearpump.experiments.yarn
 
 object Constants {
+  val CONTAINER_USER = "gearpump.yarn.user"
   val APPMASTER_NAME = "gearpump.yarn.applicationmaster.name"
   val APPMASTER_COMMAND = "gearpump.yarn.applicationmaster.command"
   val APPMASTER_MEMORY = "gearpump.yarn.applicationmaster.memory"
diff --git a/experiments/yarn/src/main/scala/org/apache/gearpump/experiments/yarn/appmaster/Command.scala b/experiments/yarn/src/main/scala/org/apache/gearpump/experiments/yarn/appmaster/Command.scala
index 711506a..fb482c4 100644
--- a/experiments/yarn/src/main/scala/org/apache/gearpump/experiments/yarn/appmaster/Command.scala
+++ b/experiments/yarn/src/main/scala/org/apache/gearpump/experiments/yarn/appmaster/Command.scala
@@ -48,7 +48,7 @@
     : String = {
     val exe = config.getString(java)
 
-    s"$exe -cp ${classPath.mkString(":")}:" +
+    s"$exe -noverify -cp ${classPath.mkString(":")}:" +
       "$CLASSPATH " + properties.mkString(" ") +
       s" $mainClazz ${cliOpts.mkString(" ")} 2>&1 | /usr/bin/tee -a ${LOG_DIR_EXPANSION_VAR}/stderr"
   }
diff --git a/experiments/yarn/src/main/scala/org/apache/gearpump/experiments/yarn/client/LaunchCluster.scala b/experiments/yarn/src/main/scala/org/apache/gearpump/experiments/yarn/client/LaunchCluster.scala
index 2475728..267d588 100644
--- a/experiments/yarn/src/main/scala/org/apache/gearpump/experiments/yarn/client/LaunchCluster.scala
+++ b/experiments/yarn/src/main/scala/org/apache/gearpump/experiments/yarn/client/LaunchCluster.scala
@@ -26,6 +26,7 @@
 import org.apache.gearpump.cluster.ClusterConfig
 import org.apache.gearpump.cluster.main.{ArgumentsParser, CLIOption, ParseResult}
 import org.apache.gearpump.experiments.yarn.Constants
+import org.apache.gearpump.experiments.yarn.Constants._
 import org.apache.gearpump.experiments.yarn.appmaster.AppMasterCommand
 import org.apache.gearpump.experiments.yarn.appmaster.YarnAppMaster.{ActiveConfig, GetActiveConfig}
 import org.apache.gearpump.experiments.yarn.glue.Records.{ApplicationId, Resource}
@@ -184,6 +185,10 @@
     if (parsed.getBoolean(VERBOSE)) {
       LogUtil.verboseLogToConsole()
     }
+    if (inputAkkaConf.hasPath(CONTAINER_USER)) {
+      val userName = inputAkkaConf.getString(CONTAINER_USER)
+      System.setProperty("HADOOP_USER_NAME", userName)
+    }
 
     val yarnConfig = new YarnConfig()
     val fs = new FileSystem(yarnConfig)
diff --git a/experiments/yarn/src/test/scala/org/apache/gearpump/experiments/yarn/appmaster/CommandSpec.scala b/experiments/yarn/src/test/scala/org/apache/gearpump/experiments/yarn/appmaster/CommandSpec.scala
deleted file mode 100644
index c4c5a65..0000000
--- a/experiments/yarn/src/test/scala/org/apache/gearpump/experiments/yarn/appmaster/CommandSpec.scala
+++ /dev/null
@@ -1,95 +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 org.apache.gearpump.experiments.yarn.appmaster
-
-import com.typesafe.config.ConfigFactory
-import org.scalatest.{BeforeAndAfterAll, FlatSpec, Matchers}
-
-import org.apache.gearpump.cluster.TestUtil
-import org.apache.gearpump.transport.HostPort
-
-class CommandSpec extends FlatSpec with Matchers with BeforeAndAfterAll {
-  val config = ConfigFactory.parseString(
-
-    """
-      |
-      |gearpump {
-      |  yarn {
-      |    client {
-      |      package -path = "/user/gearpump/gearpump.zip"
-      |    }
-      |
-      |    applicationmaster {
-      |      ## Memory of YarnAppMaster
-      |        command = "$JAVA_HOME/bin/java -Xmx512m"
-      |      memory = "512"
-      |      vcores = "1"
-      |      queue = "default"
-      |    }
-      |
-      |    master {
-      |      ## Memory of master daemon
-      |      command = "$JAVA_HOME/bin/java  -Xmx512m"
-      |      memory = "512"
-      |      vcores = "1"
-      |    }
-      |
-      |    worker {
-      |      ## memory of worker daemon
-      |      command = "$JAVA_HOME/bin/java  -Xmx512m"
-      |      containers = "4"
-      |      ## This also contains all memory for child executors.
-      |      memory = "4096"
-      |      vcores = "1"
-      |    }
-      |    services {
-      |      enabled = true
-      |    }
-      |  }
-      |}
-    """.stripMargin).withFallback(TestUtil.DEFAULT_CONFIG)
-
-  "MasterCommand" should "create correct command line" in {
-    val version = "gearpump-0.1"
-    val master = MasterCommand(config, version, HostPort("127.0.0.1", 8080))
-
-    // scalastyle:off line.size.limit
-    val expected = "$JAVA_HOME/bin/java  -Xmx512m -cp conf:pack/gearpump-0.1/conf:pack/gearpump-0.1/lib/daemon/*:pack/gearpump-0.1/lib/*:$CLASSPATH -Dgearpump.cluster.masters.0=127.0.0.1:8080 -Dgearpump.hostname=127.0.0.1 -Dgearpump.master-resource-manager-container-id={{CONTAINER_ID}} -Dgearpump.home={{LOCAL_DIRS}}/{{CONTAINER_ID}}/pack/gearpump-0.1 -Dgearpump.log.daemon.dir=<LOG_DIR> -Dgearpump.log.application.dir=<LOG_DIR> org.apache.gearpump.cluster.main.Master -ip 127.0.0.1 -port 8080 2>&1 | /usr/bin/tee -a <LOG_DIR>/stderr"
-    // scalastyle:on line.size.limit
-    assert(master.get == expected)
-  }
-
-  "WorkerCommand" should "create correct command line" in {
-    val version = "gearpump-0.1"
-    val worker = WorkerCommand(config, version, HostPort("127.0.0.1", 8080), "worker-machine")
-    // scalastyle:off line.size.limit
-    val expected = "$JAVA_HOME/bin/java  -Xmx512m -cp conf:pack/gearpump-0.1/conf:pack/gearpump-0.1/lib/daemon/*:pack/gearpump-0.1/lib/*:$CLASSPATH -Dgearpump.cluster.masters.0=127.0.0.1:8080 -Dgearpump.log.daemon.dir=<LOG_DIR> -Dgearpump.worker-resource-manager-container-id={{CONTAINER_ID}} -Dgearpump.home={{LOCAL_DIRS}}/{{CONTAINER_ID}}/pack/gearpump-0.1 -Dgearpump.log.application.dir=<LOG_DIR> -Dgearpump.hostname=worker-machine org.apache.gearpump.cluster.main.Worker  2>&1 | /usr/bin/tee -a <LOG_DIR>/stderr"
-    // scalastyle:on line.size.limit
-    assert(worker.get == expected)
-  }
-
-  "AppMasterCommand" should "create correct command line" in {
-    val version = "gearpump-0.1"
-    val appmaster = AppMasterCommand(config, version, Array("arg1", "arg2", "arg3"))
-    // scalastyle:off line.size.limit
-    val expected = "$JAVA_HOME/bin/java -Xmx512m -cp conf:pack/gearpump-0.1/conf:pack/gearpump-0.1/dashboard:pack/gearpump-0.1/lib/*:pack/gearpump-0.1/lib/daemon/*:pack/gearpump-0.1/lib/services/*:pack/gearpump-0.1/lib/yarn/*:$CLASSPATH -Dgearpump.home={{LOCAL_DIRS}}/{{CONTAINER_ID}}/pack/gearpump-0.1 -Dgearpump.binary-version-with-scala-version=gearpump-0.1 -Dgearpump.log.daemon.dir=<LOG_DIR> -Dgearpump.log.application.dir=<LOG_DIR> -Dgearpump.hostname={{NM_HOST}} org.apache.gearpump.experiments.yarn.appmaster.YarnAppMaster  arg1 arg2 arg3 2>&1 | /usr/bin/tee -a <LOG_DIR>/stderr"
-    // scalastyle:on line.size.limit
-    assert(appmaster.get == expected)
-  }
-}
\ No newline at end of file
diff --git a/experiments/yarn/src/test/scala/org/apache/gearpump/experiments/yarn/client/LaunchClusterSpec.scala b/experiments/yarn/src/test/scala/org/apache/gearpump/experiments/yarn/client/LaunchClusterSpec.scala
index c0a4ee2..3f2f9cb 100644
--- a/experiments/yarn/src/test/scala/org/apache/gearpump/experiments/yarn/client/LaunchClusterSpec.scala
+++ b/experiments/yarn/src/test/scala/org/apache/gearpump/experiments/yarn/client/LaunchClusterSpec.scala
@@ -186,7 +186,7 @@
 
     // val workerResources = ArgumentCaptor.forClass(classOf[List[Resource]])
     // scalastyle:off line.size.limit
-    val expectedCommand = "$JAVA_HOME/bin/java -Xmx512m -cp conf:pack/gearpump-0.2/conf:pack/gearpump-0.2/dashboard:pack/gearpump-0.2/lib/*:pack/gearpump-0.2/lib/daemon/*:pack/gearpump-0.2/lib/services/*:pack/gearpump-0.2/lib/yarn/*:$CLASSPATH -Dgearpump.home={{LOCAL_DIRS}}/{{CONTAINER_ID}}/pack/gearpump-0.2 -Dgearpump.binary-version-with-scala-version=gearpump-0.2 -Dgearpump.log.daemon.dir=<LOG_DIR> -Dgearpump.log.application.dir=<LOG_DIR> -Dgearpump.hostname={{NM_HOST}} org.apache.gearpump.experiments.yarn.appmaster.YarnAppMaster  -conf /root/.gearpump_application_0_0000/conf/ -package gearpump.zip 2>&1 | /usr/bin/tee -a <LOG_DIR>/stderr"
+    val expectedCommand = "$JAVA_HOME/bin/java -Xmx512m -noverify -cp conf:pack/gearpump-0.2/conf:pack/gearpump-0.2/dashboard:pack/gearpump-0.2/lib/*:pack/gearpump-0.2/lib/daemon/*:pack/gearpump-0.2/lib/services/*:pack/gearpump-0.2/lib/yarn/*:$CLASSPATH -Dgearpump.home={{LOCAL_DIRS}}/{{CONTAINER_ID}}/pack/gearpump-0.2 -Dgearpump.binary-version-with-scala-version=gearpump-0.2 -Dgearpump.log.daemon.dir=<LOG_DIR> -Dgearpump.log.application.dir=<LOG_DIR> -Dgearpump.hostname={{NM_HOST}} org.apache.gearpump.experiments.yarn.appmaster.YarnAppMaster  -conf /root/.gearpump_application_0_0000/conf/ -package gearpump.zip 2>&1 | /usr/bin/tee -a <LOG_DIR>/stderr"
     // scalastyle:on line.size.limit
     verify(yarnClient).submit("gearpump", appId, expectedCommand,
       Resource.newInstance(512, 1), "default",
diff --git a/external/hbase/src/main/scala/org/apache/gearpump/external/hbase/HBaseSink.scala b/external/hbase/src/main/scala/org/apache/gearpump/external/hbase/HBaseSink.scala
index f85c43b..4b41ba1 100644
--- a/external/hbase/src/main/scala/org/apache/gearpump/external/hbase/HBaseSink.scala
+++ b/external/hbase/src/main/scala/org/apache/gearpump/external/hbase/HBaseSink.scala
@@ -31,22 +31,22 @@
 import org.apache.hadoop.hbase.{HBaseConfiguration, TableName}
 import org.apache.hadoop.security.UserGroupInformation
 
-class HBaseSink(userconfig: UserConfig, tableName: String,
+class HBaseSink(userConfig: UserConfig, tableName: String,
     val conn: (UserConfig, Configuration)
     => Connection, @transient var configuration: Configuration)
   extends DataSink {
 
-  lazy val connection = conn(userconfig, configuration)
+  lazy val connection = conn(userConfig, configuration)
   lazy val table = connection.getTable(TableName.valueOf(tableName))
 
   override def open(context: TaskContext): Unit = {}
 
-  def this(userconfig: UserConfig, tableName: String, configuration: Configuration) = {
-    this(userconfig, tableName, HBaseSink.getConnection, configuration)
+  def this(userConfig: UserConfig, tableName: String, configuration: Configuration) = {
+    this(userConfig, tableName, HBaseSink.getConnection, configuration)
   }
 
-  def this(userconfig: UserConfig, tableName: String) = {
-    this(userconfig, tableName, HBaseConfiguration.create())
+  def this(userConfig: UserConfig, tableName: String) = {
+    this(userConfig, tableName, HBaseConfiguration.create())
   }
 
   def insert(rowKey: String, columnGroup: String, columnName: String, value: String): Unit = {
@@ -120,14 +120,14 @@
   val COLUMN_NAME = "hbase.table.column.name"
   val HBASE_USER = "hbase.user"
 
-  def apply[T](userconfig: UserConfig, tableName: String, configuration: Configuration)
+  def apply[T](userConfig: UserConfig, tableName: String, configuration: Configuration)
   : HBaseSink = {
-    new HBaseSink(userconfig, tableName, configuration)
+    new HBaseSink(userConfig, tableName, configuration)
   }
 
-  def apply[T](userconfig: UserConfig, tableName: String)
+  def apply[T](userConfig: UserConfig, tableName: String)
   : HBaseSink = {
-    new HBaseSink(userconfig, tableName)
+    new HBaseSink(userConfig, tableName)
   }
 
   private def getConnection(userConfig: UserConfig, configuration: Configuration): Connection = {
diff --git a/external/hbase/src/main/scala/org/apache/gearpump/external/hbase/dsl/HBaseDSLSink.scala b/external/hbase/src/main/scala/org/apache/gearpump/external/hbase/dsl/HBaseDSLSink.scala
index 2417763..22efa89 100644
--- a/external/hbase/src/main/scala/org/apache/gearpump/external/hbase/dsl/HBaseDSLSink.scala
+++ b/external/hbase/src/main/scala/org/apache/gearpump/external/hbase/dsl/HBaseDSLSink.scala
@@ -18,13 +18,10 @@
 package org.apache.gearpump.external.hbase.dsl
 
 import scala.language.implicitConversions
-
 import org.apache.hadoop.conf.Configuration
-
 import org.apache.gearpump.cluster.UserConfig
 import org.apache.gearpump.external.hbase.HBaseSink
-import org.apache.gearpump.streaming.dsl.Stream
-import org.apache.gearpump.streaming.dsl.Stream.Sink
+import org.apache.gearpump.streaming.dsl.scalaapi.Stream
 
 /** Create a HBase DSL Sink */
 class HBaseDSLSink[T](stream: Stream[T]) {
diff --git a/external/kafka/src/main/scala/org/apache/gearpump/streaming/kafka/dsl/KafkaDSL.scala b/external/kafka/src/main/scala/org/apache/gearpump/streaming/kafka/dsl/KafkaDSL.scala
index f1bb26a..996ae0b 100644
--- a/external/kafka/src/main/scala/org/apache/gearpump/streaming/kafka/dsl/KafkaDSL.scala
+++ b/external/kafka/src/main/scala/org/apache/gearpump/streaming/kafka/dsl/KafkaDSL.scala
@@ -21,7 +21,7 @@
 
 import org.apache.gearpump.cluster.UserConfig
 import org.apache.gearpump.streaming.dsl
-import org.apache.gearpump.streaming.dsl.StreamApp
+import org.apache.gearpump.streaming.dsl.scalaapi.{Stream, StreamApp}
 import org.apache.gearpump.streaming.kafka.{KafkaSink, KafkaSource}
 import org.apache.gearpump.streaming.transaction.api.CheckpointStoreFactory
 
@@ -44,7 +44,7 @@
       parallelism: Int = 1,
       config: UserConfig = UserConfig.empty,
       description: String = "KafkaSource"
-      ): dsl.Stream[T] = {
+      ): Stream[T] = {
     app.source[T](new KafkaSource(topics, properties), parallelism, config, description)
   }
 
@@ -66,19 +66,19 @@
       properties: Properties,
       parallelism: Int = 1,
       config: UserConfig = UserConfig.empty,
-      description: String = "KafkaSource"): dsl.Stream[T] = {
+      description: String = "KafkaSource"): Stream[T] = {
     val source = new KafkaSource(topics, properties)
     source.setCheckpointStore(checkpointStoreFactory)
     app.source[T](source, parallelism, config, description)
   }
 
   import scala.language.implicitConversions
-  implicit def streamToKafkaDSL[T](stream: dsl.Stream[T]): KafkaDSL[T] = {
+  implicit def streamToKafkaDSL[T](stream: Stream[T]): KafkaDSL[T] = {
     new KafkaDSL[T](stream)
   }
 }
 
-class KafkaDSL[T](stream: dsl.Stream[T]) {
+class KafkaDSL[T](stream: Stream[T]) {
 
   /**
    * Sinks data to Kafka
@@ -94,7 +94,7 @@
       properties: Properties,
       parallelism: Int = 1,
       userConfig: UserConfig = UserConfig.empty,
-      description: String = "KafkaSink"): dsl.Stream[T] = {
+      description: String = "KafkaSink"): Stream[T] = {
     stream.sink(new KafkaSink(topic, properties), parallelism, userConfig, description)
   }
 }
\ No newline at end of file
diff --git a/daemon/src/main/resources/META-INF/services/org.apache.gearpump.jarstore.JarStore b/gearpump-hadoop/src/main/resources/META-INF/services/org.apache.gearpump.jarstore.JarStore
similarity index 94%
rename from daemon/src/main/resources/META-INF/services/org.apache.gearpump.jarstore.JarStore
rename to gearpump-hadoop/src/main/resources/META-INF/services/org.apache.gearpump.jarstore.JarStore
index e173a8a..2809247 100644
--- a/daemon/src/main/resources/META-INF/services/org.apache.gearpump.jarstore.JarStore
+++ b/gearpump-hadoop/src/main/resources/META-INF/services/org.apache.gearpump.jarstore.JarStore
@@ -16,5 +16,4 @@
 # limitations under the License.
 #
 
-org.apache.gearpump.jarstore.local.LocalJarStore
 org.apache.gearpump.jarstore.dfs.DFSJarStore
\ No newline at end of file
diff --git a/daemon/src/main/scala/org/apache/gearpump/jarstore/dfs/DFSJarStore.scala b/gearpump-hadoop/src/main/scala/org/apache/gearpump/jarstore/dfs/DFSJarStore.scala
similarity index 99%
rename from daemon/src/main/scala/org/apache/gearpump/jarstore/dfs/DFSJarStore.scala
rename to gearpump-hadoop/src/main/scala/org/apache/gearpump/jarstore/dfs/DFSJarStore.scala
index ebaf354..ae4cf46 100644
--- a/daemon/src/main/scala/org/apache/gearpump/jarstore/dfs/DFSJarStore.scala
+++ b/gearpump-hadoop/src/main/scala/org/apache/gearpump/jarstore/dfs/DFSJarStore.scala
@@ -18,12 +18,11 @@
 package org.apache.gearpump.jarstore.dfs
 
 import java.io.{InputStream, OutputStream}
-
-import com.typesafe.config.Config
-import org.apache.gearpump.jarstore.JarStore
 import org.apache.gearpump.util.Constants
+import org.apache.gearpump.jarstore.JarStore
 import org.apache.hadoop.conf.Configuration
 import org.apache.hadoop.fs.Path
+import com.typesafe.config.Config
 import org.apache.hadoop.fs.permission.{FsAction, FsPermission}
 
 /**
diff --git a/integrationtest/autorun/autorun.sh b/integrationtest/autorun/autorun.sh
index b63e10f..2d97ba0 100644
--- a/integrationtest/autorun/autorun.sh
+++ b/integrationtest/autorun/autorun.sh
@@ -1,7 +1,20 @@
 #!/bin/bash
-
-# Licensed under the Apache License, Version 2.0
-# See accompanying LICENSE file.
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
 
 RECIPIENTS_FILE="recipients.txt"
 CHECK_INTERVAL="1m"
diff --git a/integrationtest/core/src/it/scala/org/apache/gearpump/integrationtest/TestSpecBase.scala b/integrationtest/core/src/it/scala/org/apache/gearpump/integrationtest/TestSpecBase.scala
index 1e49236..a00495a 100644
--- a/integrationtest/core/src/it/scala/org/apache/gearpump/integrationtest/TestSpecBase.scala
+++ b/integrationtest/core/src/it/scala/org/apache/gearpump/integrationtest/TestSpecBase.scala
@@ -51,7 +51,7 @@
   lazy val commandLineClient = cluster.commandLineClient
   lazy val restClient = cluster.restClient
 
-  lazy val wordCountJar = cluster.queryBuiltInExampleJars("wordcount-").head
+  lazy val wordCountJar = cluster.queryBuiltInExampleJars("wordcount_").head
   lazy val wordCountName = "wordCount"
   lazy val wordCountClass = "org.apache.gearpump.streaming.examples.wordcount.WordCount"
 
diff --git a/integrationtest/core/src/it/scala/org/apache/gearpump/integrationtest/checklist/CommandLineSpec.scala b/integrationtest/core/src/it/scala/org/apache/gearpump/integrationtest/checklist/CommandLineSpec.scala
index f4e463e..3fa6f6a 100644
--- a/integrationtest/core/src/it/scala/org/apache/gearpump/integrationtest/checklist/CommandLineSpec.scala
+++ b/integrationtest/core/src/it/scala/org/apache/gearpump/integrationtest/checklist/CommandLineSpec.scala
@@ -90,7 +90,7 @@
       // setup
       val args = "-debug true -sleep 10"
       val appId = expectSubmitAppSuccess(wordCountJar, args)
-      var success = commandLineClient.killApp(appId)
+      val success = commandLineClient.killApp(appId)
       success shouldBe true
     }
 
diff --git a/integrationtest/core/src/it/scala/org/apache/gearpump/integrationtest/checklist/ConnectorKafkaSpec.scala b/integrationtest/core/src/it/scala/org/apache/gearpump/integrationtest/checklist/ConnectorKafkaSpec.scala
index d8bdc1e..e69a425 100644
--- a/integrationtest/core/src/it/scala/org/apache/gearpump/integrationtest/checklist/ConnectorKafkaSpec.scala
+++ b/integrationtest/core/src/it/scala/org/apache/gearpump/integrationtest/checklist/ConnectorKafkaSpec.scala
@@ -28,7 +28,7 @@
 class ConnectorKafkaSpec extends TestSpecBase {
 
   private lazy val kafkaCluster = new KafkaCluster(cluster.getNetworkGateway)
-  private lazy val kafkaJar = cluster.queryBuiltInExampleJars("kafka-").head
+  private lazy val kafkaJar = cluster.queryBuiltInExampleJars("kafka").head
   private var producer: NumericalDataProducer = null
 
   override def beforeAll(): Unit = {
diff --git a/integrationtest/core/src/it/scala/org/apache/gearpump/integrationtest/checklist/DynamicDagSpec.scala b/integrationtest/core/src/it/scala/org/apache/gearpump/integrationtest/checklist/DynamicDagSpec.scala
index 89b8ef7..c4276bd 100644
--- a/integrationtest/core/src/it/scala/org/apache/gearpump/integrationtest/checklist/DynamicDagSpec.scala
+++ b/integrationtest/core/src/it/scala/org/apache/gearpump/integrationtest/checklist/DynamicDagSpec.scala
@@ -26,14 +26,14 @@
 
   val sourceTaskClass = "org.apache.gearpump.streaming.examples.sol.SOLStreamProducer"
   val sinkTaskClass = "org.apache.gearpump.streaming.examples.sol.SOLStreamProcessor"
-  lazy val solJar = cluster.queryBuiltInExampleJars("sol-").head
+  lazy val solJar = cluster.queryBuiltInExampleJars("sol").head
 
   "dynamic dag" should {
     "can retrieve a list of built-in partitioner classes" in {
       val partitioners = restClient.queryBuiltInPartitioners()
       partitioners.length should be > 0
       partitioners.foreach(clazz =>
-        clazz should startWith("org.apache.gearpump.partitioner.")
+        clazz should startWith("org.apache.gearpump.streaming.partitioner.")
       )
     }
 
diff --git a/integrationtest/core/src/it/scala/org/apache/gearpump/integrationtest/checklist/ExampleSpec.scala b/integrationtest/core/src/it/scala/org/apache/gearpump/integrationtest/checklist/ExampleSpec.scala
index 5554a26..97d5daf 100644
--- a/integrationtest/core/src/it/scala/org/apache/gearpump/integrationtest/checklist/ExampleSpec.scala
+++ b/integrationtest/core/src/it/scala/org/apache/gearpump/integrationtest/checklist/ExampleSpec.scala
@@ -32,7 +32,7 @@
 
   "distributed shell" should {
     "execute commands on machines where its executors are running" in {
-      val distShellJar = cluster.queryBuiltInExampleJars("distributedshell-").head
+      val distShellJar = cluster.queryBuiltInExampleJars("distributedshell").head
       val mainClass = "org.apache.gearpump.examples.distributedshell.DistributedShell"
       val clientClass = "org.apache.gearpump.examples.distributedshell.DistributedShellClient"
       val appId = restClient.getNextAvailableAppId()
@@ -61,7 +61,7 @@
   }
 
   "wordcount" should {
-    val wordCountJarNamePrefix = "wordcount-"
+    val wordCountJarNamePrefix = "wordcount_"
     behave like streamingApplication(wordCountJarNamePrefix, wordCountName)
 
     "can submit immediately after killing a former one" in {
@@ -84,19 +84,19 @@
   }
 
   "wordcount(java)" should {
-    val wordCountJavaJarNamePrefix = "wordcountjava-"
+    val wordCountJavaJarNamePrefix = "wordcountjava"
     val wordCountJavaName = "wordcountJava"
     behave like streamingApplication(wordCountJavaJarNamePrefix, wordCountJavaName)
   }
 
   "sol" should {
-    val solJarNamePrefix = "sol-"
+    val solJarNamePrefix = "sol"
     val solName = "sol"
     behave like streamingApplication(solJarNamePrefix, solName)
   }
 
   "complexdag" should {
-    val dynamicDagJarNamePrefix = "complexdag-"
+    val dynamicDagJarNamePrefix = "complexdag"
     val dynamicDagName = "dag"
     behave like streamingApplication(dynamicDagJarNamePrefix, dynamicDagName)
   }
diff --git a/integrationtest/core/src/it/scala/org/apache/gearpump/integrationtest/checklist/MessageDeliverySpec.scala b/integrationtest/core/src/it/scala/org/apache/gearpump/integrationtest/checklist/MessageDeliverySpec.scala
index bb9982a..9186364 100644
--- a/integrationtest/core/src/it/scala/org/apache/gearpump/integrationtest/checklist/MessageDeliverySpec.scala
+++ b/integrationtest/core/src/it/scala/org/apache/gearpump/integrationtest/checklist/MessageDeliverySpec.scala
@@ -64,7 +64,7 @@
               "-sourceTask", sourcePartitionNum).mkString(" ")
             val appId = restClient.getNextAvailableAppId()
 
-            val stateJar = cluster.queryBuiltInExampleJars("state-").head
+            val stateJar = cluster.queryBuiltInExampleJars("state").head
             val success = restClient.submitApp(stateJar, executorNum = 1, args = args)
             success shouldBe true
 
diff --git a/integrationtest/core/src/it/scala/org/apache/gearpump/integrationtest/checklist/StormCompatibilitySpec.scala b/integrationtest/core/src/it/scala/org/apache/gearpump/integrationtest/checklist/StormCompatibilitySpec.scala
index b327bf4..7c04cc0 100644
--- a/integrationtest/core/src/it/scala/org/apache/gearpump/integrationtest/checklist/StormCompatibilitySpec.scala
+++ b/integrationtest/core/src/it/scala/org/apache/gearpump/integrationtest/checklist/StormCompatibilitySpec.scala
@@ -53,7 +53,7 @@
   }
 
   def getStormJar(stormVersion: String): String = {
-    cluster.queryBuiltInITJars(s"storm$stormVersion-").head
+    cluster.queryBuiltInITJars(s"storm$stormVersion").head
   }
 
   "Storm over Gearpump" should withStorm {
@@ -168,11 +168,10 @@
                 s"executor $executorToKill killed")
 
               // verify no message loss
-              val detector = new
-                  MessageLossDetector(producer.lastWriteNum)
-              val kafkaReader = new
-                  SimpleKafkaReader(detector, sinkTopic, host = kafkaCluster.advertisedHost,
-                    port = kafkaCluster.advertisedPort)
+              val detector = new MessageLossDetector(producer.lastWriteNum)
+              val kafkaReader =
+                new SimpleKafkaReader(detector, sinkTopic, host = kafkaCluster.advertisedHost,
+                  port = kafkaCluster.advertisedPort)
 
               Util.retryUntil(() => {
                 kafkaReader.read()
diff --git a/licenses/LICENSE-AnchorJS.txt b/licenses/LICENSE-AnchorJS.txt
deleted file mode 100644
index d5266e6..0000000
--- a/licenses/LICENSE-AnchorJS.txt
+++ /dev/null
@@ -1,21 +0,0 @@
-# The MIT License (MIT)
-
-Copyright (c) 2015 Bryan Braun
-
-Permission is hereby granted, free of charge, to any person obtaining a copy
-of this software and associated documentation files (the "Software"), to deal
-in the Software without restriction, including without limitation the rights
-to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
-copies of the Software, and to permit persons to whom the Software is
-furnished to do so, subject to the following conditions:
-
-The above copyright notice and this permission notice shall be included in
-all copies or substantial portions of the Software.
-
-THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
-IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
-FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
-AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
-LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
-OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN
-THE SOFTWARE.
diff --git a/licenses/LICENSE-activation.txt b/licenses/LICENSE-activation.txt
new file mode 100644
index 0000000..cd0c8d5
--- /dev/null
+++ b/licenses/LICENSE-activation.txt
@@ -0,0 +1,93 @@
+COMMON DEVELOPMENT AND DISTRIBUTION LICENSE (CDDL) Version 1.0 1. 
+
+Definitions. 
+
+1.1. Contributor means each individual or entity that creates or contributes to the creation of Modifications. 
+
+1.2. Contributor Version means the combination of the Original Software, prior Modifications used by a Contributor (if any), and the Modifications made by that particular Contributor. 
+
+1.3. Covered Software means (a) the Original Software, or (b) Modifications, or (c) the combination of files containing Original Software with files containing Modifications, in each case including portions thereof. 
+
+1.4. Executable means the Covered Software in any form other than Source Code. 
+
+1.5. Initial Developer means the individual or entity that first makes Original Software available under this License. 
+
+1.6. Larger Work means a work which combines Covered Software or portions thereof with code not governed by the terms of this License. 
+
+1.7. License means this document. 
+
+1.8. Licensable means having the right to grant, to the maximum extent possible, whether at the time of the initial grant or subsequently acquired, any and all of the rights conveyed herein. 
+
+1.9. Modifications means the Source Code and Executable form of any of the following: A. Any file that results from an addition to, deletion from or modification of the contents of a file containing Original Software or previous Modifications; B. Any new file that contains any part of the Original Software or previous Modification; or C. Any new file that is contributed or otherwise made available under the terms of this License. 
+
+1.10. Original Software means the Source Code and Executable form of computer software code that is originally released under this License. 
+
+1.11. Patent Claims means any patent claim(s), now owned or hereafter acquired, including without limitation, method, process, and apparatus claims, in any patent Licensable by grantor. 
+
+1.12. Source Code means (a) the common form of computer software code in which modifications are made and (b) associated documentation included in or with such code. 
+
+1.13. You (or Your) means an individual or a legal entity exercising rights under, and complying with all of the terms of, this License. For legal entities, You includes any entity which controls, is controlled by, or is under common control with You. For purposes of this definition, control means (a) the power, direct or indirect, to cause the direction or management of such entity, whether by contract or otherwise, or (b) ownership of more than fifty percent (50%) of the outstanding shares or beneficial ownership of such entity. 
+
+2. License Grants.
+
+ 2.1. The Initial Developer Grant. Conditioned upon Your compliance with Section 3.1 below and subject to third party intellectual property claims, the Initial Developer hereby grants You a world-wide, royalty-free, non-exclusive license: 
+
+(a) under intellectual property rights (other than patent or trademark) Licensable by Initial Developer, to use, reproduce, modify, display, perform, sublicense and distribute the Original Software (or portions thereof), with or without Modifications, and/or as part of a Larger Work; and 
+
+(b) under Patent Claims infringed by the making, using or selling of Original Software, to make, have made, use, practice, sell, and offer for sale, and/or otherwise dispose of the Original Software (or portions thereof);
+
+ (c) The licenses granted in Sections 2.1(a) and (b) are effective on the date Initial Developer first distributes or otherwise makes the Original Software available to a third party under the terms of this License;
+
+ (d) Notwithstanding Section 2.1(b) above, no patent license is granted: (1) for code that You delete from the Original Software, or (2) for infringements caused by: (i) the modification of the Original Software, or (ii) the combination of the Original Software with other software or devices. 
+
+2.2. Contributor Grant. Conditioned upon Your compliance with Section 3.1 below and subject to third party intellectual property claims, each Contributor hereby grants You a world-wide, royalty-free, non-exclusive license: 
+
+(a) under intellectual property rights (other than patent or trademark) Licensable by Contributor to use, reproduce, modify, display, perform, sublicense and distribute the Modifications created by such Contributor (or portions thereof), either on an unmodified basis, with other Modifications, as Covered Software and/or as part of a Larger Work; and 
+
+(b) under Patent Claims infringed by the making, using, or selling of Modifications made by that Contributor either alone and/or in combination with its Contributor Version (or portions of such combination), to make, use, sell, offer for sale, have made, and/or otherwise dispose of: (1) Modifications made by that Contributor (or portions thereof); and (2) the combination of Modifications made by that Contributor with its Contributor Version (or portions of such combination). 
+
+(c) The licenses granted in Sections 2.2(a) and 2.2(b) are effective on the date Contributor first distributes or otherwise makes the Modifications available to a third party.
+
+(d) Notwithstanding Section 2.2(b) above, no patent license is granted: (1) for any code that Contributor has deleted from the Contributor Version; (2) for infringements caused by: (i) third party modifications of Contributor Version, or (ii) the combination of Modifications made by that Contributor with other software (except as part of the Contributor Version) or other devices; or (3) under Patent Claims infringed by Covered Software in the absence of Modifications made by that Contributor. 
+
+3. Distribution Obligations. 
+
+3.1. Availability of Source Code. Any Covered Software that You distribute or otherwise make available in Executable form must also be made available in Source Code form and that Source Code form must be distributed only under the terms of this License. You must include a copy of this License with every copy of the Source Code form of the Covered Software You distribute or otherwise make available. You must inform recipients of any such Covered Software in Executable form as to how they can obtain such Covered Software in Source Code form in a reasonable manner on or through a medium customarily used for software exchange. 
+
+3.2. Modifications. The Modifications that You create or to which You contribute are governed by the terms of this License. You represent that You believe Your Modifications are Your original creation(s) and/or You have sufficient rights to grant the rights conveyed by this License. 
+
+3.3. Required Notices. You must include a notice in each of Your Modifications that identifies You as the Contributor of the Modification. You may not remove or alter any copyright, patent or trademark notices contained within the Covered Software, or any notices of licensing or any descriptive text giving attribution to any Contributor or the Initial Developer. 
+
+3.4. Application of Additional Terms. You may not offer or impose any terms on any Covered Software in Source Code form that alters or restricts the applicable version of this License or the recipients rights hereunder. You may choose to offer, and to charge a fee for, warranty, support, indemnity or liability obligations to one or more recipients of Covered Software. However, you may do so only on Your own behalf, and not on behalf of the Initial Developer or any Contributor. You must make it absolutely clear that any such warranty, support, indemnity or liability obligation is offered by You alone, and You hereby agree to indemnify the Initial Developer and every Contributor for any liability incurred by the Initial Developer or such Contributor as a result of warranty, support, indemnity or liability terms You offer.
+
+3.5. Distribution of Executable Versions. You may distribute the Executable form of the Covered Software under the terms of this License or under the terms of a license of Your choice, which may contain terms different from this License, provided that You are in compliance with the terms of this License and that the license for the Executable form does not attempt to limit or alter the recipients rights in the Source Code form from the rights set forth in this License. If You distribute the Covered Software in Executable form under a different license, You must make it absolutely clear that any terms which differ from this License are offered by You alone, not by the Initial Developer or Contributor. You hereby agree to indemnify the Initial Developer and every Contributor for any liability incurred by the Initial Developer or such Contributor as a result of any such terms You offer. 
+
+3.6. Larger Works. You may create a Larger Work by combining Covered Software with other code not governed by the terms of this License and distribute the Larger Work as a single product. In such a case, You must make sure the requirements of this License are fulfilled for the Covered Software. 
+
+4. Versions of the License. 
+
+4.1. New Versions. Sun Microsystems, Inc. is the initial license steward and may publish revised and/or new versions of this License from time to time. Each version will be given a distinguishing version number. Except as provided in Section 4.3, no one other than the license steward has the right to modify this License. 
+
+4.2. Effect of New Versions. You may always continue to use, distribute or otherwise make the Covered Software available under the terms of the version of the License under which You originally received the Covered Software. If the Initial Developer includes a notice in the Original Software prohibiting it from being distributed or otherwise made available under any subsequent version of the License, You must distribute and make the Covered Software available under the terms of the version of the License under which You originally received the Covered Software. Otherwise, You may also choose to use, distribute or otherwise make the Covered Software available under the terms of any subsequent version of the License published by the license steward. 
+
+4.3. Modified Versions. When You are an Initial Developer and You want to create a new license for Your Original Software, You may create and use a modified version of this License if You: (a) rename the license and remove any references to the name of the license steward (except to note that the license differs from this License); and (b) otherwise make it clear that the license contains terms which differ from this License. 
+
+5. DISCLAIMER OF WARRANTY. COVERED SOFTWARE IS PROVIDED UNDER THIS LICENSE ON AN AS IS BASIS, WITHOUT WARRANTY OF ANY KIND, EITHER EXPRESSED OR IMPLIED, INCLUDING, WITHOUT LIMITATION, WARRANTIES THAT THE COVERED SOFTWARE IS FREE OF DEFECTS, MERCHANTABLE, FIT FOR A PARTICULAR PURPOSE OR NON-INFRINGING. THE ENTIRE RISK AS TO THE QUALITY AND PERFORMANCE OF THE COVERED SOFTWARE IS WITH YOU. SHOULD ANY COVERED SOFTWARE PROVE DEFECTIVE IN ANY RESPECT, YOU (NOT THE INITIAL DEVELOPER OR ANY OTHER CONTRIBUTOR) ASSUME THE COST OF ANY NECESSARY SERVICING, REPAIR OR CORRECTION. THIS DISCLAIMER OF WARRANTY CONSTITUTES AN ESSENTIAL PART OF THIS LICENSE. NO USE OF ANY COVERED SOFTWARE IS AUTHORIZED HEREUNDER EXCEPT UNDER THIS DISCLAIMER. 
+
+6. TERMINATION. 
+
+6.1. This License and the rights granted hereunder will terminate automatically if You fail to comply with terms herein and fail to cure such breach within 30 days of becoming aware of the breach. Provisions which, by their nature, must remain in effect beyond the termination of this License shall survive. 
+
+6.2. If You assert a patent infringement claim (excluding declaratory judgment actions) against Initial Developer or a Contributor (the Initial Developer or Contributor against whom You assert such claim is referred to as Participant) alleging that the Participant Software (meaning the Contributor Version where the Participant is a Contributor or the Original Software where the Participant is the Initial Developer) directly or indirectly infringes any patent, then any and all rights granted directly or indirectly to You by such Participant, the Initial Developer (if the Initial Developer is not the Participant) and all Contributors under Sections 2.1 and/or 2.2 of this License shall, upon 60 days notice from Participant terminate prospectively and automatically at the expiration of such 60 day notice period, unless if within such 60 day period You withdraw Your claim with respect to the Participant Software against such Participant either unilaterally or pursuant to a written agreement with Participant. 
+
+6.3. In the event of termination under Sections 6.1 or 6.2 above, all end user licenses that have been validly granted by You or any distributor hereunder prior to termination (excluding licenses granted to You by any distributor) shall survive termination. 
+
+7. LIMITATION OF LIABILITY. UNDER NO CIRCUMSTANCES AND UNDER NO LEGAL THEORY, WHETHER TORT (INCLUDING NEGLIGENCE), CONTRACT, OR OTHERWISE, SHALL YOU, THE INITIAL DEVELOPER, ANY OTHER CONTRIBUTOR, OR ANY DISTRIBUTOR OF COVERED SOFTWARE, OR ANY SUPPLIER OF ANY OF SUCH PARTIES, BE LIABLE TO ANY PERSON FOR ANY INDIRECT, SPECIAL, INCIDENTAL, OR CONSEQUENTIAL DAMAGES OF ANY CHARACTER INCLUDING, WITHOUT LIMITATION, DAMAGES FOR LOST PROFITS, LOSS OF GOODWILL, WORK STOPPAGE, COMPUTER FAILURE OR MALFUNCTION, OR ANY AND ALL OTHER COMMERCIAL DAMAGES OR LOSSES, EVEN IF SUCH PARTY SHALL HAVE BEEN INFORMED OF THE POSSIBILITY OF SUCH DAMAGES. THIS LIMITATION OF LIABILITY SHALL NOT APPLY TO LIABILITY FOR DEATH OR PERSONAL INJURY RESULTING FROM SUCH PARTYS NEGLIGENCE TO THE EXTENT APPLICABLE LAW PROHIBITS SUCH LIMITATION. SOME JURISDICTIONS DO NOT ALLOW THE EXCLUSION OR LIMITATION OF INCIDENTAL OR CONSEQUENTIAL DAMAGES, SO THIS EXCLUSION AND LIMITATION MAY NOT APPLY TO YOU. 
+
+8. U.S. GOVERNMENT END USERS. The Covered Software is a commercial item, as that term is defined in 48 C.F.R. 2.101 (Oct. 1995), consisting of commercial computer software (as that term is defined at 48 C.F.R.  252.227-7014(a)(1)) and commercial computer software documentation as such terms are used in 48 C.F.R. 12.212 (Sept. 1995). Consistent with 48 C.F.R. 12.212 and 48 C.F.R. 227.7202-1 through 227.7202-4 (June 1995), all U.S. Government End Users acquire Covered Software with only those rights set forth herein. This U.S. Government Rights clause is in lieu of, and supersedes, any other FAR, DFAR, or other clause or provision that addresses Government rights in computer software under this License. 
+
+9. MISCELLANEOUS. This License represents the complete agreement concerning subject matter hereof. If any provision of this License is held to be unenforceable, such provision shall be reformed only to the extent necessary to make it enforceable. This License shall be governed by the law of the jurisdiction specified in a notice contained within the Original Software (except to the extent applicable law, if any, provides otherwise), excluding such jurisdictions conflict-of-law provisions. Any litigation relating to this License shall be subject to the jurisdiction of the courts located in the jurisdiction and venue specified in a notice contained within the Original Software, with the losing party responsible for costs, including, without limitation, court costs and reasonable attorneys fees and expenses. The application of the United Nations Convention on Contracts for the International Sale of Goods is expressly excluded. Any law or regulation which provides that the language of a contract shall be construed against the drafter shall not apply to this License. You agree that You alone are responsible for compliance with the United States export administration regulations (and the export control laws and regulation of any other countries) when You use, distribute or otherwise make available any Covered Software. 
+
+10. RESPONSIBILITY FOR CLAIMS. As between Initial Developer and the Contributors, each party is responsible for claims and damages arising, directly or indirectly, out of its utilization of rights under this License and You agree to work with Initial Developer and Contributors to distribute such responsibility on an equitable basis. Nothing herein is intended or shall be deemed to constitute any admission of liability. 
+
+NOTICE PURSUANT TO SECTION 9 OF THE COMMON DEVELOPMENT AND DISTRIBUTION LICENSE (CDDL) The code released under the CDDL shall be governed by the laws of the State of California (excluding conflict-of-law provisions). Any litigation relating to this License shall be subject to the jurisdiction of the Federal Courts of the Northern District of California and the state courts of the State of California, with venue lying in Santa Clara County, California.
diff --git a/licenses/LICENSE-angular-loading-bar.txt b/licenses/LICENSE-angular-loading-bar.txt
new file mode 100644
index 0000000..252c23a
--- /dev/null
+++ b/licenses/LICENSE-angular-loading-bar.txt
@@ -0,0 +1,20 @@
+The MIT License (MIT)
+
+Copyright (c) 2013-2014 Wes Cruver
+
+Permission is hereby granted, free of charge, to any person obtaining a copy of
+this software and associated documentation files (the "Software"), to deal in
+the Software without restriction, including without limitation the rights to
+use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of
+the Software, and to permit persons to whom the Software is furnished to do so,
+subject to the following conditions:
+
+The above copyright notice and this permission notice shall be included in all
+copies or substantial portions of the Software.
+
+THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, FITNESS
+FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR
+COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER
+IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN
+CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE.
diff --git a/licenses/LICENSE-angular-motion.txt b/licenses/LICENSE-angular-motion.txt
new file mode 100644
index 0000000..472ac23
--- /dev/null
+++ b/licenses/LICENSE-angular-motion.txt
@@ -0,0 +1,8 @@
+MIT License
+Copyright (c) <year> <copyright holders>
+
+Permission is hereby granted, free of charge, to any person obtaining a copy of this software and associated documentation files (the "Software"), to deal in the Software without restriction, including without limitation the rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and to permit persons to whom the Software is furnished to do so, subject to the following conditions:
+
+The above copyright notice and this permission notice shall be included in all copies or substantial portions of the Software.
+
+THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE.
diff --git a/licenses/LICENSE-angular-smart-table.txt b/licenses/LICENSE-angular-smart-table.txt
new file mode 100644
index 0000000..89a65eb
--- /dev/null
+++ b/licenses/LICENSE-angular-smart-table.txt
@@ -0,0 +1,7 @@
+Copyright (C) 2016 Laurent Renard.
+
+Permission is hereby granted, free of charge, to any person obtaining a copy of this software and associated documentation files (the "Software"), to deal in the Software without restriction, including without limitation the rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and to permit persons to whom the Software is furnished to do so, subject to the following conditions:
+
+The above copyright notice and this permission notice shall be included in all copies or substantial portions of the Software.
+
+THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE.
diff --git a/licenses/LICENSE-angular-strap.txt b/licenses/LICENSE-angular-strap.txt
new file mode 100644
index 0000000..fd3d26d
--- /dev/null
+++ b/licenses/LICENSE-angular-strap.txt
@@ -0,0 +1,9 @@
+The MIT License
+
+Copyright (c) 2012-2014 Olivier Louvignes http://olouv.com
+
+Permission is hereby granted, free of charge, to any person obtaining a copy of this software and associated documentation files (the "Software"), to deal in the Software without restriction, including without limitation the rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and to permit persons to whom the Software is furnished to do so, subject to the following conditions:
+
+The above copyright notice and this permission notice shall be included in all copies or substantial portions of the Software.
+
+THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE.
diff --git a/licenses/LICENSE-angular-touch.txt b/licenses/LICENSE-angular-touch.txt
new file mode 100644
index 0000000..8ee066e
--- /dev/null
+++ b/licenses/LICENSE-angular-touch.txt
@@ -0,0 +1,9 @@
+The MIT License
+
+Copyright (c) 2010-2015 Google, Inc. http://angularjs.org
+
+Permission is hereby granted, free of charge, to any person obtaining a copy of this software and associated documentation files (the "Software"), to deal in the Software without restriction, including without limitation the rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and to permit persons to whom the Software is furnished to do so, subject to the following conditions:
+
+The above copyright notice and this permission notice shall be included in all copies or substantial portions of the Software.
+
+THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE.
diff --git a/licenses/LICENSE-normalize.txt b/licenses/LICENSE-angular-ui-router.txt
similarity index 92%
copy from licenses/LICENSE-normalize.txt
copy to licenses/LICENSE-angular-ui-router.txt
index ce68d94..6413b09 100644
--- a/licenses/LICENSE-normalize.txt
+++ b/licenses/LICENSE-angular-ui-router.txt
@@ -1,6 +1,6 @@
-# The MIT License (MIT)
+The MIT License
 
-Copyright © Nicolas Gallagher and Jonathan Neal
+Copyright (c) 2013-2015 The AngularUI Team, Karsten Sperling
 
 Permission is hereby granted, free of charge, to any person obtaining a copy
 of this software and associated documentation files (the "Software"), to deal
@@ -18,4 +18,4 @@
 AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
 LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
 OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN
-THE SOFTWARE.
\ No newline at end of file
+THE SOFTWARE.
diff --git a/licenses/LICENSE-angular.txt b/licenses/LICENSE-angular.txt
new file mode 100644
index 0000000..05fc838
--- /dev/null
+++ b/licenses/LICENSE-angular.txt
@@ -0,0 +1,9 @@
+The MIT License (MIT)
+
+Copyright (c) 2016 Angular
+
+Permission is hereby granted, free of charge, to any person obtaining a copy of this software and associated documentation files (the "Software"), to deal in the Software without restriction, including without limitation the rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and to permit persons to whom the Software is furnished to do so, subject to the following conditions:
+
+The above copyright notice and this permission notice shall be included in all copies or substantial portions of the Software.
+
+THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE.
diff --git a/licenses/LICENSE-normalize.txt b/licenses/LICENSE-angularjs.txt
similarity index 92%
rename from licenses/LICENSE-normalize.txt
rename to licenses/LICENSE-angularjs.txt
index ce68d94..b8de5aa 100644
--- a/licenses/LICENSE-normalize.txt
+++ b/licenses/LICENSE-angularjs.txt
@@ -1,6 +1,6 @@
-# The MIT License (MIT)
+The MIT License
 
-Copyright © Nicolas Gallagher and Jonathan Neal
+Copyright (c) 2010-2016 Google, Inc. http://angularjs.org
 
 Permission is hereby granted, free of charge, to any person obtaining a copy
 of this software and associated documentation files (the "Software"), to deal
@@ -18,4 +18,5 @@
 AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
 LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
 OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN
-THE SOFTWARE.
\ No newline at end of file
+THE SOFTWARE.
+
diff --git a/licenses/LICENSE-aopalliance.txt b/licenses/LICENSE-aopalliance.txt
new file mode 100644
index 0000000..0c2090e
--- /dev/null
+++ b/licenses/LICENSE-aopalliance.txt
@@ -0,0 +1,4 @@
+The AOP Alliance project is a joint open-source project between several software engineering people who are interested in AOP and Java.
+
+
+LICENCE: all the source code provided by AOP Alliance is Public Domain.
diff --git a/licenses/LICENSE-asm.txt b/licenses/LICENSE-asm.txt
new file mode 100644
index 0000000..c5aba7b
--- /dev/null
+++ b/licenses/LICENSE-asm.txt
@@ -0,0 +1,29 @@
+Copyright (c) 2000-2011 INRIA, France Telecom
+All rights reserved.
+
+Redistribution and use in source and binary forms, with or without
+modification, are permitted provided that the following conditions
+are met:
+
+1. Redistributions of source code must retain the above copyright
+   notice, this list of conditions and the following disclaimer.
+
+2. Redistributions in binary form must reproduce the above copyright
+   notice, this list of conditions and the following disclaimer in the
+   documentation and/or other materials provided with the distribution.
+
+3. Neither the name of the copyright holders nor the names of its
+   contributors may be used to endorse or promote products derived from
+   this software without specific prior written permission.
+
+THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS"
+AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE
+IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE
+ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE
+LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR
+CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF
+SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS
+INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN
+CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE)
+ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF
+THE POSSIBILITY OF SUCH DAMAGE.
diff --git a/licenses/LICENSE-bootstrap-additions.txt b/licenses/LICENSE-bootstrap-additions.txt
new file mode 100644
index 0000000..2d59292
--- /dev/null
+++ b/licenses/LICENSE-bootstrap-additions.txt
@@ -0,0 +1,9 @@
+The MIT License
+
+Copyright (c) 2014 Olivier Louvignes http://olouv.com
+
+Permission is hereby granted, free of charge, to any person obtaining a copy of this software and associated documentation files (the "Software"), to deal in the Software without restriction, including without limitation the rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and to permit persons to whom the Software is furnished to do so, subject to the following conditions:
+
+The above copyright notice and this permission notice shall be included in all copies or substantial portions of the Software.
+
+THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE.
diff --git a/licenses/LICENSE-bootstrap.txt b/licenses/LICENSE-bootstrap.txt
index 8d363b6..4e32163 100644
--- a/licenses/LICENSE-bootstrap.txt
+++ b/licenses/LICENSE-bootstrap.txt
@@ -1,6 +1,7 @@
-# The MIT License (MIT)
+The MIT License (MIT)
 
 Copyright (c) 2011-2016 Twitter, Inc.
+Copyright (c) 2011-2016 The Bootstrap Authors
 
 Permission is hereby granted, free of charge, to any person obtaining a copy
 of this software and associated documentation files (the "Software"), to deal
diff --git a/licenses/LICENSE-clipboard.js.txt b/licenses/LICENSE-clipboard.js.txt
new file mode 100644
index 0000000..2d6176f
--- /dev/null
+++ b/licenses/LICENSE-clipboard.js.txt
@@ -0,0 +1,7 @@
+Copyright © 2016 Zeno Rocha <hi@zenorocha.com>
+
+Permission is hereby granted, free of charge, to any person obtaining a copy of this software and associated documentation files (the “Software”), to deal in the Software without restriction, including without limitation the rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and to permit persons to whom the Software is furnished to do so, subject to the following conditions:
+
+The above copyright notice and this permission notice shall be included in all copies or substantial portions of the Software.
+
+THE SOFTWARE IS PROVIDED “AS IS”, WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE.
diff --git a/licenses/LICENSE-clojure.txt b/licenses/LICENSE-clojure.txt
new file mode 100644
index 0000000..d120419
--- /dev/null
+++ b/licenses/LICENSE-clojure.txt
@@ -0,0 +1,3 @@
+The use and distribution terms for this software are covered by the Eclipse Public License 1.0, which can be found in the file epl-v10.html at the root of this distribution. By using this software in any fashion, you are agreeing to be bound by the terms of this license. You must not remove this notice, or any other, from this software.
+
+
diff --git a/licenses/LICENSE-d3js.txt b/licenses/LICENSE-d3js.txt
new file mode 100644
index 0000000..721bd22
--- /dev/null
+++ b/licenses/LICENSE-d3js.txt
@@ -0,0 +1,27 @@
+Copyright 2010-2016 Mike Bostock
+All rights reserved.
+
+Redistribution and use in source and binary forms, with or without modification,
+are permitted provided that the following conditions are met:
+
+* Redistributions of source code must retain the above copyright notice, this
+  list of conditions and the following disclaimer.
+
+* Redistributions in binary form must reproduce the above copyright notice,
+  this list of conditions and the following disclaimer in the documentation
+  and/or other materials provided with the distribution.
+
+* Neither the name of the author nor the names of contributors may be used to
+  endorse or promote products derived from this software without specific prior
+  written permission.
+
+THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND
+ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED
+WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE
+DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE LIABLE FOR
+ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES
+(INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES;
+LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON
+ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS
+SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
diff --git a/licenses/LICENSE-derive.txt b/licenses/LICENSE-derive.txt
new file mode 100644
index 0000000..5f0ccbe
--- /dev/null
+++ b/licenses/LICENSE-derive.txt
@@ -0,0 +1,7 @@
+Copyright (c) 2016 haoyi.sg@gmail.com
+
+Permission is hereby granted, free of charge, to any person obtaining a copy of this software and associated documentation files (the "Software"), to deal in the Software without restriction, including without limitation the rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and to permit persons to whom the Software is furnished to do so, subject to the following conditions:
+
+The above copyright notice and this permission notice shall be included in all copies or substantial portions of the Software.
+
+THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE.
diff --git a/licenses/LICENSE-font-awesome.txt b/licenses/LICENSE-font-awesome.txt
new file mode 100644
index 0000000..6d0813b
--- /dev/null
+++ b/licenses/LICENSE-font-awesome.txt
@@ -0,0 +1,11 @@
+License
+
+The Font Awesome font is licensed under the SIL OFL 1.1:
+http://scripts.sil.org/OFL
+Font Awesome CSS, LESS, and Sass files are licensed under the MIT License:
+https://opensource.org/licenses/mit-license.html
+The Font Awesome documentation is licensed under the CC BY 3.0 License:
+http://creativecommons.org/licenses/by/3.0/
+Attribution is no longer required as of Font Awesome 3.0, but much appreciated:
+Font Awesome by Dave Gandy - http://fontawesome.io
+Full details: http://fontawesome.io/license/
diff --git a/licenses/LICENSE-glyphicons.txt b/licenses/LICENSE-glyphicons.txt
deleted file mode 100644
index 6a1d292..0000000
--- a/licenses/LICENSE-glyphicons.txt
+++ /dev/null
@@ -1,21 +0,0 @@
-# The MIT License (MIT)
-
-Copyright (c) 2010 - 2016 Jan Kovarik
-
-Permission is hereby granted, free of charge, to any person obtaining a copy
-of this software and associated documentation files (the "Software"), to deal
-in the Software without restriction, including without limitation the rights
-to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
-copies of the Software, and to permit persons to whom the Software is
-furnished to do so, subject to the following conditions:
-
-The above copyright notice and this permission notice shall be included in
-all copies or substantial portions of the Software.
-
-THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
-IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
-FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
-AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
-LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
-OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN
-THE SOFTWARE.
diff --git a/licenses/LICENSE-htmlcleaner.txt b/licenses/LICENSE-htmlcleaner.txt
new file mode 100644
index 0000000..23aed7a
--- /dev/null
+++ b/licenses/LICENSE-htmlcleaner.txt
@@ -0,0 +1,33 @@
+HtmlCleaner is distributed under BSD License. It gives the freedom for anyone to use, explore, modify, and distribute HtmlCleaner, but without any warranty.
+
+    Copyright (c) 2006-2016, HtmlCleaner team.
+    All rights reserved.
+    
+    Redistribution and use of this software in source and binary forms, 
+    with or without modification, are permitted provided that the 
+    following conditions are met:
+    
+    * Redistributions of source code must retain the above
+      copyright notice, this list of conditions and the
+      following disclaimer.
+    
+    * Redistributions in binary form must reproduce the above
+      copyright notice, this list of conditions and the
+      following disclaimer in the documentation and/or other
+      materials provided with the distribution.
+    
+    * The name of HtmlCleaner may not be used to endorse or promote
+      products derived from this software without specific prior
+      written permission.
+    
+    THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS 
+    "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT 
+    LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR 
+    A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT 
+    OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, 
+    SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT 
+    LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, 
+    DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY 
+    THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT 
+    (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE 
+    OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
diff --git a/licenses/LICENSE-jawn-parser.txt b/licenses/LICENSE-jawn-parser.txt
new file mode 100644
index 0000000..c41fae0
--- /dev/null
+++ b/licenses/LICENSE-jawn-parser.txt
@@ -0,0 +1,8 @@
+The MIT License (MIT)
+Copyright Erik Osheim, 2012-2016.
+
+Permission is hereby granted, free of charge, to any person obtaining a copy of this software and associated documentation files (the "Software"), to deal in the Software without restriction, including without limitation the rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and to permit persons to whom the Software is furnished to do so, subject to the following conditions:
+
+The above copyright notice and this permission notice shall be included in all copies or substantial portions of the Software.
+
+THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE.
diff --git a/licenses/LICENSE-jaxb-api.txt b/licenses/LICENSE-jaxb-api.txt
new file mode 100644
index 0000000..1507ec7
--- /dev/null
+++ b/licenses/LICENSE-jaxb-api.txt
@@ -0,0 +1,210 @@
+COMMON DEVELOPMENT AND DISTRIBUTION LICENSE (CDDL)Version 1.1
+
+1. Definitions.
+
+1.1. “Contributor” means each individual or entity that creates or contributes to the creation of Modifications.
+1.2. “Contributor Version” means the combination of the Original Software, prior Modifications used by a Contributor (if any), and the Modifications made by that particular Contributor.
+1.3. “Covered Software” means (a) the Original Software, or (b) Modifications, or (c) the combination of files containing Original Software with files containing Modifications, in each case including portions thereof.
+1.4. “Executable” means the Covered Software in any form other than Source Code.
+1.5. “Initial Developer” means the individual or entity that first makes Original Software available under this License.
+1.6. “Larger Work” means a work which combines Covered Software or portions thereof with code not governed by the terms of this License.
+1.7. “License” means this document.
+1.8. “Licensable” means having the right to grant, to the maximum extent possible, whether at the time of the initial grant or subsequently acquired, any and all of the rights conveyed herein.
+1.9. “Modifications” means the Source Code and Executable form of any of the following:
+A. Any file that results from an addition to, deletion from or modification of the contents of a file containing Original Software or previous Modifications;
+B. Any new file that contains any part of the Original Software or previous Modification; or
+C. Any new file that is contributed or otherwise made available under the terms of this License.
+1.10. “Original Software” means the Source Code and Executable form of computer software code that is originally released under this License.
+1.11. “Patent Claims” means any patent claim(s), now owned or hereafter acquired, including without limitation, method, process, and apparatus claims, in any patent Licensable by grantor.
+1.12. “Source Code” means (a) the common form of computer software code in which modifications are made and (b) associated documentation included in or with such code.
+1.13. “You” (or “Your”) means an individual or a legal entity exercising rights under, and complying with all of the terms of, this License. For legal entities, “You” includes any entity which controls, is controlled by, or is under common control with You. For purposes of this definition, “control” means (a) the power, direct or indirect, to cause the direction or management of such entity, whether by contract or otherwise, or (b) ownership of more than fifty percent (50%) of the outstanding shares or beneficial ownership of such entity.
+2. License Grants.
+
+2.1. The Initial Developer Grant.
+Conditioned upon Your compliance with Section 3.1 below and subject to third party intellectual property claims, the Initial Developer hereby grants You a world-wide, royalty-free, non-exclusive license:
+(a) under intellectual property rights (other than patent or trademark) Licensable by Initial Developer, to use, reproduce, modify, display, perform, sublicense and distribute the Original Software (or portions thereof), with or without Modifications, and/or as part of a Larger Work; and
+(b) under Patent Claims infringed by the making, using or selling of Original Software, to make, have made, use, practice, sell, and offer for sale, and/or otherwise dispose of the Original Software (or portions thereof).
+(c) The licenses granted in Sections 2.1(a) and (b) are effective on the date Initial Developer first distributes or otherwise makes the Original Software available to a third party under the terms of this License.
+(d) Notwithstanding Section 2.1(b) above, no patent license is granted: (1) for code that You delete from the Original Software, or (2) for infringements caused by: (i) the modification of the Original Software, or (ii) the combination of the Original Software with other software or devices.
+2.2. Contributor Grant.
+Conditioned upon Your compliance with Section 3.1 below and subject to third party intellectual property claims, each Contributor hereby grants You a world-wide, royalty-free, non-exclusive license:
+(a) under intellectual property rights (other than patent or trademark) Licensable by Contributor to use, reproduce, modify, display, perform, sublicense and distribute the Modifications created by such Contributor (or portions thereof), either on an unmodified basis, with other Modifications, as Covered Software and/or as part of a Larger Work; and
+(b) under Patent Claims infringed by the making, using, or selling of Modifications made by that Contributor either alone and/or in combination with its Contributor Version (or portions of such combination), to make, use, sell, offer for sale, have made, and/or otherwise dispose of: (1) Modifications made by that Contributor (or portions thereof); and (2) the combination of Modifications made by that Contributor with its Contributor Version (or portions of such combination).
+(c) The licenses granted in Sections 2.2(a) and 2.2(b) are effective on the date Contributor first distributes or otherwise makes the Modifications available to a third party.
+(d) Notwithstanding Section 2.2(b) above, no patent license is granted: (1) for any code that Contributor has deleted from the Contributor Version; (2) for infringements caused by: (i) third party modifications of Contributor Version, or (ii) the combination of Modifications made by that Contributor with other software (except as part of the Contributor Version) or other devices; or (3) under Patent Claims infringed by Covered Software in the absence of Modifications made by that Contributor.
+3. Distribution Obligations.
+
+3.1. Availability of Source Code.
+Any Covered Software that You distribute or otherwise make available in Executable form must also be made available in Source Code form and that Source Code form must be distributed only under the terms of this License. You must include a copy of this License with every copy of the Source Code form of the Covered Software You distribute or otherwise make available. You must inform recipients of any such Covered Software in Executable form as to how they can obtain such Covered Software in Source Code form in a reasonable manner on or through a medium customarily used for software exchange.
+3.2. Modifications.
+The Modifications that You create or to which You contribute are governed by the terms of this License. You represent that You believe Your Modifications are Your original creation(s) and/or You have sufficient rights to grant the rights conveyed by this License.
+3.3. Required Notices.
+You must include a notice in each of Your Modifications that identifies You as the Contributor of the Modification. You may not remove or alter any copyright, patent or trademark notices contained within the Covered Software, or any notices of licensing or any descriptive text giving attribution to any Contributor or the Initial Developer.
+3.4. Application of Additional Terms.
+You may not offer or impose any terms on any Covered Software in Source Code form that alters or restricts the applicable version of this License or the recipients' rights hereunder. You may choose to offer, and to charge a fee for, warranty, support, indemnity or liability obligations to one or more recipients of Covered Software. However, you may do so only on Your own behalf, and not on behalf of the Initial Developer or any Contributor. You must make it absolutely clear that any such warranty, support, indemnity or liability obligation is offered by You alone, and You hereby agree to indemnify the Initial Developer and every Contributor for any liability incurred by the Initial Developer or such Contributor as a result of warranty, support, indemnity or liability terms You offer.
+3.5. Distribution of Executable Versions.
+You may distribute the Executable form of the Covered Software under the terms of this License or under the terms of a license of Your choice, which may contain terms different from this License, provided that You are in compliance with the terms of this License and that the license for the Executable form does not attempt to limit or alter the recipient's rights in the Source Code form from the rights set forth in this License. If You distribute the Covered Software in Executable form under a different license, You must make it absolutely clear that any terms which differ from this License are offered by You alone, not by the Initial Developer or Contributor. You hereby agree to indemnify the Initial Developer and every Contributor for any liability incurred by the Initial Developer or such Contributor as a result of any such terms You offer.
+3.6. Larger Works.
+You may create a Larger Work by combining Covered Software with other code not governed by the terms of this License and distribute the Larger Work as a single product. In such a case, You must make sure the requirements of this License are fulfilled for the Covered Software.
+4. Versions of the License.
+
+4.1. New Versions.
+Oracle is the initial license steward and may publish revised and/or new versions of this License from time to time. Each version will be given a distinguishing version number. Except as provided in Section 4.3, no one other than the license steward has the right to modify this License.
+4.2. Effect of New Versions.
+You may always continue to use, distribute or otherwise make the Covered Software available under the terms of the version of the License under which You originally received the Covered Software. If the Initial Developer includes a notice in the Original Software prohibiting it from being distributed or otherwise made available under any subsequent version of the License, You must distribute and make the Covered Software available under the terms of the version of the License under which You originally received the Covered Software. Otherwise, You may also choose to use, distribute or otherwise make the Covered Software available under the terms of any subsequent version of the License published by the license steward.
+4.3. Modified Versions.
+When You are an Initial Developer and You want to create a new license for Your Original Software, You may create and use a modified version of this License if You: (a) rename the license and remove any references to the name of the license steward (except to note that the license differs from this License); and (b) otherwise make it clear that the license contains terms which differ from this License.
+5. DISCLAIMER OF WARRANTY.
+
+COVERED SOFTWARE IS PROVIDED UNDER THIS LICENSE ON AN “AS IS” BASIS, WITHOUT WARRANTY OF ANY KIND, EITHER EXPRESSED OR IMPLIED, INCLUDING, WITHOUT LIMITATION, WARRANTIES THAT THE COVERED SOFTWARE IS FREE OF DEFECTS, MERCHANTABLE, FIT FOR A PARTICULAR PURPOSE OR NON-INFRINGING. THE ENTIRE RISK AS TO THE QUALITY AND PERFORMANCE OF THE COVERED SOFTWARE IS WITH YOU. SHOULD ANY COVERED SOFTWARE PROVE DEFECTIVE IN ANY RESPECT, YOU (NOT THE INITIAL DEVELOPER OR ANY OTHER CONTRIBUTOR) ASSUME THE COST OF ANY NECESSARY SERVICING, REPAIR OR CORRECTION. THIS DISCLAIMER OF WARRANTY CONSTITUTES AN ESSENTIAL PART OF THIS LICENSE. NO USE OF ANY COVERED SOFTWARE IS AUTHORIZED HEREUNDER EXCEPT UNDER THIS DISCLAIMER.
+
+6. TERMINATION.
+
+6.1. This License and the rights granted hereunder will terminate automatically if You fail to comply with terms herein and fail to cure such breach within 30 days of becoming aware of the breach. Provisions which, by their nature, must remain in effect beyond the termination of this License shall survive.
+6.2. If You assert a patent infringement claim (excluding declaratory judgment actions) against Initial Developer or a Contributor (the Initial Developer or Contributor against whom You assert such claim is referred to as “Participant”) alleging that the Participant Software (meaning the Contributor Version where the Participant is a Contributor or the Original Software where the Participant is the Initial Developer) directly or indirectly infringes any patent, then any and all rights granted directly or indirectly to You by such Participant, the Initial Developer (if the Initial Developer is not the Participant) and all Contributors under Sections 2.1 and/or 2.2 of this License shall, upon 60 days notice from Participant terminate prospectively and automatically at the expiration of such 60 day notice period, unless if within such 60 day period You withdraw Your claim with respect to the Participant Software against such Participant either unilaterally or pursuant to a written agreement with Participant.
+6.3. If You assert a patent infringement claim against Participant alleging that the Participant Software directly or indirectly infringes any patent where such claim is resolved (such as by license or settlement) prior to the initiation of patent infringement litigation, then the reasonable value of the licenses granted by such Participant under Sections 2.1 or 2.2 shall be taken into account in determining the amount or value of any payment or license.
+6.4. In the event of termination under Sections 6.1 or 6.2 above, all end user licenses that have been validly granted by You or any distributor hereunder prior to termination (excluding licenses granted to You by any distributor) shall survive termination.
+7. LIMITATION OF LIABILITY.
+
+UNDER NO CIRCUMSTANCES AND UNDER NO LEGAL THEORY, WHETHER TORT (INCLUDING NEGLIGENCE), CONTRACT, OR OTHERWISE, SHALL YOU, THE INITIAL DEVELOPER, ANY OTHER CONTRIBUTOR, OR ANY DISTRIBUTOR OF COVERED SOFTWARE, OR ANY SUPPLIER OF ANY OF SUCH PARTIES, BE LIABLE TO ANY PERSON FOR ANY INDIRECT, SPECIAL, INCIDENTAL, OR CONSEQUENTIAL DAMAGES OF ANY CHARACTER INCLUDING, WITHOUT LIMITATION, DAMAGES FOR LOSS OF GOODWILL, WORK STOPPAGE, COMPUTER FAILURE OR MALFUNCTION, OR ANY AND ALL OTHER COMMERCIAL DAMAGES OR LOSSES, EVEN IF SUCH PARTY SHALL HAVE BEEN INFORMED OF THE POSSIBILITY OF SUCH DAMAGES. THIS LIMITATION OF LIABILITY SHALL NOT APPLY TO LIABILITY FOR DEATH OR PERSONAL INJURY RESULTING FROM SUCH PARTY'S NEGLIGENCE TO THE EXTENT APPLICABLE LAW PROHIBITS SUCH LIMITATION. SOME JURISDICTIONS DO NOT ALLOW THE EXCLUSION OR LIMITATION OF INCIDENTAL OR CONSEQUENTIAL DAMAGES, SO THIS EXCLUSION AND LIMITATION MAY NOT APPLY TO YOU.
+
+8. U.S. GOVERNMENT END USERS.
+
+The Covered Software is a “commercial item,” as that term is defined in 48 C.F.R. 2.101 (Oct. 1995), consisting of “commercial computer software” (as that term is defined at 48 C.F.R. § 252.227-7014(a)(1)) and “commercial computer software documentation” as such terms are used in 48 C.F.R. 12.212 (Sept. 1995). Consistent with 48 C.F.R. 12.212 and 48 C.F.R. 227.7202-1 through 227.7202-4 (June 1995), all U.S. Government End Users acquire Covered Software with only those rights set forth herein. This U.S. Government Rights clause is in lieu of, and supersedes, any other FAR, DFAR, or other clause or provision that addresses Government rights in computer software under this License.
+
+9. MISCELLANEOUS.
+
+This License represents the complete agreement concerning subject matter hereof. If any provision of this License is held to be unenforceable, such provision shall be reformed only to the extent necessary to make it enforceable. This License shall be governed by the law of the jurisdiction specified in a notice contained within the Original Software (except to the extent applicable law, if any, provides otherwise), excluding such jurisdiction's conflict-of-law provisions. Any litigation relating to this License shall be subject to the jurisdiction of the courts located in the jurisdiction and venue specified in a notice contained within the Original Software, with the losing party responsible for costs, including, without limitation, court costs and reasonable attorneys' fees and expenses. The application of the United Nations Convention on Contracts for the International Sale of Goods is expressly excluded. Any law or regulation which provides that the language of a contract shall be construed against the drafter shall not apply to this License. You agree that You alone are responsible for compliance with the United States export administration regulations (and the export control laws and regulation of any other countries) when You use, distribute or otherwise make available any Covered Software.
+
+10. RESPONSIBILITY FOR CLAIMS.
+
+As between Initial Developer and the Contributors, each party is responsible for claims and damages arising, directly or indirectly, out of its utilization of rights under this License and You agree to work with Initial Developer and Contributors to distribute such responsibility on an equitable basis. Nothing herein is intended or shall be deemed to constitute any admission of liability.
+
+NOTICE PURSUANT TO SECTION 9 OF THE COMMON DEVELOPMENT AND DISTRIBUTION LICENSE (CDDL)
+
+The code released under the CDDL shall be governed by the laws of the State of California (excluding conflict-of-law provisions). Any litigation relating to this License shall be subject to the jurisdiction of the Federal Courts of the Northern District of California and the state courts of the State of California, with venue lying in Santa Clara County, California.
+
+
+
+The GNU General Public License (GPL) Version 2, June 1991
+
+Copyright (C) 1989, 1991 Free Software Foundation, Inc.
+59 Temple Place, Suite 330, Boston, MA 02111-1307 USA
+
+Everyone is permitted to copy and distribute verbatim copies
+of this license document, but changing it is not allowed.
+
+Preamble
+
+The licenses for most software are designed to take away your freedom to share and change it. By contrast, the GNU General Public License is intended to guarantee your freedom to share and change free software--to make sure the software is free for all its users. This General Public License applies to most of the Free Software Foundation's software and to any other program whose authors commit to using it. (Some other Free Software Foundation software is covered by the GNU Library General Public License instead.) You can apply it to your programs, too.
+
+When we speak of free software, we are referring to freedom, not price. Our General Public Licenses are designed to make sure that you have the freedom to distribute copies of free software (and charge for this service if you wish), that you receive source code or can get it if you want it, that you can change the software or use pieces of it in new free programs; and that you know you can do these things.
+
+To protect your rights, we need to make restrictions that forbid anyone to deny you these rights or to ask you to surrender the rights. These restrictions translate to certain responsibilities for you if you distribute copies of the software, or if you modify it.
+
+For example, if you distribute copies of such a program, whether gratis or for a fee, you must give the recipients all the rights that you have. You must make sure that they, too, receive or can get the source code. And you must show them these terms so they know their rights.
+
+We protect your rights with two steps: (1) copyright the software, and (2) offer you this license which gives you legal permission to copy, distribute and/or modify the software.
+
+Also, for each author's protection and ours, we want to make certain that everyone understands that there is no warranty for this free software. If the software is modified by someone else and passed on, we want its recipients to know that what they have is not the original, so that any problems introduced by others will not reflect on the original authors' reputations.
+
+Finally, any free program is threatened constantly by software patents. We wish to avoid the danger that redistributors of a free program will individually obtain patent licenses, in effect making the program proprietary. To prevent this, we have made it clear that any patent must be licensed for everyone's free use or not licensed at all.
+
+The precise terms and conditions for copying, distribution and modification follow.
+
+TERMS AND CONDITIONS FOR COPYING, DISTRIBUTION AND MODIFICATION
+
+0. This License applies to any program or other work which contains a notice placed by the copyright holder saying it may be distributed under the terms of this General Public License. The "Program", below, refers to any such program or work, and a "work based on the Program" means either the Program or any derivative work under copyright law: that is to say, a work containing the Program or a portion of it, either verbatim or with modifications and/or translated into another language. (Hereinafter, translation is included without limitation in the term "modification".) Each licensee is addressed as "you".
+
+Activities other than copying, distribution and modification are not covered by this License; they are outside its scope. The act of running the Program is not restricted, and the output from the Program is covered only if its contents constitute a work based on the Program (independent of having been made by running the Program). Whether that is true depends on what the Program does.
+
+1. You may copy and distribute verbatim copies of the Program's source code as you receive it, in any medium, provided that you conspicuously and appropriately publish on each copy an appropriate copyright notice and disclaimer of warranty; keep intact all the notices that refer to this License and to the absence of any warranty; and give any other recipients of the Program a copy of this License along with the Program.
+
+You may charge a fee for the physical act of transferring a copy, and you may at your option offer warranty protection in exchange for a fee.
+
+2. You may modify your copy or copies of the Program or any portion of it, thus forming a work based on the Program, and copy and distribute such modifications or work under the terms of Section 1 above, provided that you also meet all of these conditions:
+
+a) You must cause the modified files to carry prominent notices stating that you changed the files and the date of any change.
+b) You must cause any work that you distribute or publish, that in whole or in part contains or is derived from the Program or any part thereof, to be licensed as a whole at no charge to all third parties under the terms of this License.
+c) If the modified program normally reads commands interactively when run, you must cause it, when started running for such interactive use in the most ordinary way, to print or display an announcement including an appropriate copyright notice and a notice that there is no warranty (or else, saying that you provide a warranty) and that users may redistribute the program under these conditions, and telling the user how to view a copy of this License. (Exception: if the Program itself is interactive but does not normally print such an announcement, your work based on the Program is not required to print an announcement.)
+These requirements apply to the modified work as a whole. If identifiable sections of that work are not derived from the Program, and can be reasonably considered independent and separate works in themselves, then this License, and its terms, do not apply to those sections when you distribute them as separate works. But when you distribute the same sections as part of a whole which is a work based on the Program, the distribution of the whole must be on the terms of this License, whose permissions for other licensees extend to the entire whole, and thus to each and every part regardless of who wrote it.
+
+Thus, it is not the intent of this section to claim rights or contest your rights to work written entirely by you; rather, the intent is to exercise the right to control the distribution of derivative or collective works based on the Program.
+
+In addition, mere aggregation of another work not based on the Program with the Program (or with a work based on the Program) on a volume of a storage or distribution medium does not bring the other work under the scope of this License.
+
+3. You may copy and distribute the Program (or a work based on it, under Section 2) in object code or executable form under the terms of Sections 1 and 2 above provided that you also do one of the following:
+
+a) Accompany it with the complete corresponding machine-readable source code, which must be distributed under the terms of Sections 1 and 2 above on a medium customarily used for software interchange; or,
+b) Accompany it with a written offer, valid for at least three years, to give any third party, for a charge no more than your cost of physically performing source distribution, a complete machine-readable copy of the corresponding source code, to be distributed under the terms of Sections 1 and 2 above on a medium customarily used for software interchange; or,
+c) Accompany it with the information you received as to the offer to distribute corresponding source code. (This alternative is allowed only for noncommercial distribution and only if you received the program in object code or executable form with such an offer, in accord with Subsection b above.)
+The source code for a work means the preferred form of the work for making modifications to it. For an executable work, complete source code means all the source code for all modules it contains, plus any associated interface definition files, plus the scripts used to control compilation and installation of the executable. However, as a special exception, the source code distributed need not include anything that is normally distributed (in either source or binary form) with the major components (compiler, kernel, and so on) of the operating system on which the executable runs, unless that component itself accompanies the executable.
+
+If distribution of executable or object code is made by offering access to copy from a designated place, then offering equivalent access to copy the source code from the same place counts as distribution of the source code, even though third parties are not compelled to copy the source along with the object code.
+
+4. You may not copy, modify, sublicense, or distribute the Program except as expressly provided under this License. Any attempt otherwise to copy, modify, sublicense or distribute the Program is void, and will automatically terminate your rights under this License. However, parties who have received copies, or rights, from you under this License will not have their licenses terminated so long as such parties remain in full compliance.
+
+5. You are not required to accept this License, since you have not signed it. However, nothing else grants you permission to modify or distribute the Program or its derivative works. These actions are prohibited by law if you do not accept this License. Therefore, by modifying or distributing the Program (or any work based on the Program), you indicate your acceptance of this License to do so, and all its terms and conditions for copying, distributing or modifying the Program or works based on it.
+
+6. Each time you redistribute the Program (or any work based on the Program), the recipient automatically receives a license from the original licensor to copy, distribute or modify the Program subject to these terms and conditions. You may not impose any further restrictions on the recipients' exercise of the rights granted herein. You are not responsible for enforcing compliance by third parties to this License.
+
+7. If, as a consequence of a court judgment or allegation of patent infringement or for any other reason (not limited to patent issues), conditions are imposed on you (whether by court order, agreement or otherwise) that contradict the conditions of this License, they do not excuse you from the conditions of this License. If you cannot distribute so as to satisfy simultaneously your obligations under this License and any other pertinent obligations, then as a consequence you may not distribute the Program at all. For example, if a patent license would not permit royalty-free redistribution of the Program by all those who receive copies directly or indirectly through you, then the only way you could satisfy both it and this License would be to refrain entirely from distribution of the Program.
+
+If any portion of this section is held invalid or unenforceable under any particular circumstance, the balance of the section is intended to apply and the section as a whole is intended to apply in other circumstances.
+
+It is not the purpose of this section to induce you to infringe any patents or other property right claims or to contest validity of any such claims; this section has the sole purpose of protecting the integrity of the free software distribution system, which is implemented by public license practices. Many people have made generous contributions to the wide range of software distributed through that system in reliance on consistent application of that system; it is up to the author/donor to decide if he or she is willing to distribute software through any other system and a licensee cannot impose that choice.
+
+This section is intended to make thoroughly clear what is believed to be a consequence of the rest of this License.
+
+8. If the distribution and/or use of the Program is restricted in certain countries either by patents or by copyrighted interfaces, the original copyright holder who places the Program under this License may add an explicit geographical distribution limitation excluding those countries, so that distribution is permitted only in or among countries not thus excluded. In such case, this License incorporates the limitation as if written in the body of this License.
+
+9. The Free Software Foundation may publish revised and/or new versions of the General Public License from time to time. Such new versions will be similar in spirit to the present version, but may differ in detail to address new problems or concerns.
+
+Each version is given a distinguishing version number. If the Program specifies a version number of this License which applies to it and "any later version", you have the option of following the terms and conditions either of that version or of any later version published by the Free Software Foundation. If the Program does not specify a version number of this License, you may choose any version ever published by the Free Software Foundation.
+
+10. If you wish to incorporate parts of the Program into other free programs whose distribution conditions are different, write to the author to ask for permission. For software which is copyrighted by the Free Software Foundation, write to the Free Software Foundation; we sometimes make exceptions for this. Our decision will be guided by the two goals of preserving the free status of all derivatives of our free software and of promoting the sharing and reuse of software generally.
+
+NO WARRANTY
+
+11. BECAUSE THE PROGRAM IS LICENSED FREE OF CHARGE, THERE IS NO WARRANTY FOR THE PROGRAM, TO THE EXTENT PERMITTED BY APPLICABLE LAW. EXCEPT WHEN OTHERWISE STATED IN WRITING THE COPYRIGHT HOLDERS AND/OR OTHER PARTIES PROVIDE THE PROGRAM "AS IS" WITHOUT WARRANTY OF ANY KIND, EITHER EXPRESSED OR IMPLIED, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE. THE ENTIRE RISK AS TO THE QUALITY AND PERFORMANCE OF THE PROGRAM IS WITH YOU. SHOULD THE PROGRAM PROVE DEFECTIVE, YOU ASSUME THE COST OF ALL NECESSARY SERVICING, REPAIR OR CORRECTION.
+
+12. IN NO EVENT UNLESS REQUIRED BY APPLICABLE LAW OR AGREED TO IN WRITING WILL ANY COPYRIGHT HOLDER, OR ANY OTHER PARTY WHO MAY MODIFY AND/OR REDISTRIBUTE THE PROGRAM AS PERMITTED ABOVE, BE LIABLE TO YOU FOR DAMAGES, INCLUDING ANY GENERAL, SPECIAL, INCIDENTAL OR CONSEQUENTIAL DAMAGES ARISING OUT OF THE USE OR INABILITY TO USE THE PROGRAM (INCLUDING BUT NOT LIMITED TO LOSS OF DATA OR DATA BEING RENDERED INACCURATE OR LOSSES SUSTAINED BY YOU OR THIRD PARTIES OR A FAILURE OF THE PROGRAM TO OPERATE WITH ANY OTHER PROGRAMS), EVEN IF SUCH HOLDER OR OTHER PARTY HAS BEEN ADVISED OF THE POSSIBILITY OF SUCH DAMAGES.
+
+END OF TERMS AND CONDITIONS
+
+How to Apply These Terms to Your New Programs
+
+If you develop a new program, and you want it to be of the greatest possible use to the public, the best way to achieve this is to make it free software which everyone can redistribute and change under these terms.
+
+To do so, attach the following notices to the program. It is safest to attach them to the start of each source file to most effectively convey the exclusion of warranty; and each file should have at least the "copyright" line and a pointer to where the full notice is found.
+
+One line to give the program's name and a brief idea of what it does.
+Copyright (C) <year> <name of author>
+This program is free software; you can redistribute it and/or modify it under the terms of the GNU General Public License as published by the Free Software Foundation; either version 2 of the License, or (at your option) any later version.
+This program is distributed in the hope that it will be useful, but WITHOUT ANY WARRANTY; without even the implied warranty of MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the GNU General Public License for more details.
+You should have received a copy of the GNU General Public License along with this program; if not, write to the Free Software Foundation, Inc., 59 Temple Place, Suite 330, Boston, MA 02111-1307 USA
+Also add information on how to contact you by electronic and paper mail.
+
+If the program is interactive, make it output a short notice like this when it starts in an interactive mode:
+
+Gnomovision version 69, Copyright (C) year name of author Gnomovision comes with ABSOLUTELY NO WARRANTY; for details type `show w'. This is free software, and you are welcome to redistribute it under certain conditions; type `show c' for details.
+The hypothetical commands `show w' and `show c' should show the appropriate parts of the General Public License. Of course, the commands you use may be called something other than `show w' and `show c'; they could even be mouse-clicks or menu items--whatever suits your program.
+
+You should also get your employer (if you work as a programmer) or your school, if any, to sign a "copyright disclaimer" for the program, if necessary. Here is a sample; alter the names:
+
+Yoyodyne, Inc., hereby disclaims all copyright interest in the program `Gnomovision' (which makes passes at compilers) written by James Hacker.
+signature of Ty Coon, 1 April 1989
+Ty Coon, President of Vice
+This General Public License does not permit incorporating your program into proprietary programs. If your program is a subroutine library, you may consider it more useful to permit linking proprietary applications with the library. If this is what you want to do, use the GNU Library General Public License instead of this License.
+
+"CLASSPATH" EXCEPTION TO THE GPL VERSION 2
+
+Certain source files distributed by Oracle are subject to the following clarification and special exception to the GPL Version 2, but only where Oracle has expressly included in the particular source file's header the words "Oracle designates this particular file as subject to the "Classpath" exception as provided by Oracle in the License file that accompanied this code." 
+
+Linking this library statically or dynamically with other modules is making a combined work based on this library.  Thus, the terms and conditions of the GNU General Public License Version 2 cover the whole combination. 
+
+As a special exception, the copyright holders of this library give you permission to link this library with independent modules to produce an executable, regardless of the license terms of these independent modules, and to copy and distribute the resulting executable under terms of your choice, provided that you also meet, for each linked independent module, the terms and conditions of the license of that module.  An independent module is a module which is not derived from or based on this library.  If you modify this library, you may extend this exception to your version of the library, but you are not obligated to do so.  If you do not wish to do so, delete this exception statement from your version.
+
+
+
+
diff --git a/licenses/LICENSE-jaxb-impl.txt b/licenses/LICENSE-jaxb-impl.txt
new file mode 100644
index 0000000..1507ec7
--- /dev/null
+++ b/licenses/LICENSE-jaxb-impl.txt
@@ -0,0 +1,210 @@
+COMMON DEVELOPMENT AND DISTRIBUTION LICENSE (CDDL)Version 1.1
+
+1. Definitions.
+
+1.1. “Contributor” means each individual or entity that creates or contributes to the creation of Modifications.
+1.2. “Contributor Version” means the combination of the Original Software, prior Modifications used by a Contributor (if any), and the Modifications made by that particular Contributor.
+1.3. “Covered Software” means (a) the Original Software, or (b) Modifications, or (c) the combination of files containing Original Software with files containing Modifications, in each case including portions thereof.
+1.4. “Executable” means the Covered Software in any form other than Source Code.
+1.5. “Initial Developer” means the individual or entity that first makes Original Software available under this License.
+1.6. “Larger Work” means a work which combines Covered Software or portions thereof with code not governed by the terms of this License.
+1.7. “License” means this document.
+1.8. “Licensable” means having the right to grant, to the maximum extent possible, whether at the time of the initial grant or subsequently acquired, any and all of the rights conveyed herein.
+1.9. “Modifications” means the Source Code and Executable form of any of the following:
+A. Any file that results from an addition to, deletion from or modification of the contents of a file containing Original Software or previous Modifications;
+B. Any new file that contains any part of the Original Software or previous Modification; or
+C. Any new file that is contributed or otherwise made available under the terms of this License.
+1.10. “Original Software” means the Source Code and Executable form of computer software code that is originally released under this License.
+1.11. “Patent Claims” means any patent claim(s), now owned or hereafter acquired, including without limitation, method, process, and apparatus claims, in any patent Licensable by grantor.
+1.12. “Source Code” means (a) the common form of computer software code in which modifications are made and (b) associated documentation included in or with such code.
+1.13. “You” (or “Your”) means an individual or a legal entity exercising rights under, and complying with all of the terms of, this License. For legal entities, “You” includes any entity which controls, is controlled by, or is under common control with You. For purposes of this definition, “control” means (a) the power, direct or indirect, to cause the direction or management of such entity, whether by contract or otherwise, or (b) ownership of more than fifty percent (50%) of the outstanding shares or beneficial ownership of such entity.
+2. License Grants.
+
+2.1. The Initial Developer Grant.
+Conditioned upon Your compliance with Section 3.1 below and subject to third party intellectual property claims, the Initial Developer hereby grants You a world-wide, royalty-free, non-exclusive license:
+(a) under intellectual property rights (other than patent or trademark) Licensable by Initial Developer, to use, reproduce, modify, display, perform, sublicense and distribute the Original Software (or portions thereof), with or without Modifications, and/or as part of a Larger Work; and
+(b) under Patent Claims infringed by the making, using or selling of Original Software, to make, have made, use, practice, sell, and offer for sale, and/or otherwise dispose of the Original Software (or portions thereof).
+(c) The licenses granted in Sections 2.1(a) and (b) are effective on the date Initial Developer first distributes or otherwise makes the Original Software available to a third party under the terms of this License.
+(d) Notwithstanding Section 2.1(b) above, no patent license is granted: (1) for code that You delete from the Original Software, or (2) for infringements caused by: (i) the modification of the Original Software, or (ii) the combination of the Original Software with other software or devices.
+2.2. Contributor Grant.
+Conditioned upon Your compliance with Section 3.1 below and subject to third party intellectual property claims, each Contributor hereby grants You a world-wide, royalty-free, non-exclusive license:
+(a) under intellectual property rights (other than patent or trademark) Licensable by Contributor to use, reproduce, modify, display, perform, sublicense and distribute the Modifications created by such Contributor (or portions thereof), either on an unmodified basis, with other Modifications, as Covered Software and/or as part of a Larger Work; and
+(b) under Patent Claims infringed by the making, using, or selling of Modifications made by that Contributor either alone and/or in combination with its Contributor Version (or portions of such combination), to make, use, sell, offer for sale, have made, and/or otherwise dispose of: (1) Modifications made by that Contributor (or portions thereof); and (2) the combination of Modifications made by that Contributor with its Contributor Version (or portions of such combination).
+(c) The licenses granted in Sections 2.2(a) and 2.2(b) are effective on the date Contributor first distributes or otherwise makes the Modifications available to a third party.
+(d) Notwithstanding Section 2.2(b) above, no patent license is granted: (1) for any code that Contributor has deleted from the Contributor Version; (2) for infringements caused by: (i) third party modifications of Contributor Version, or (ii) the combination of Modifications made by that Contributor with other software (except as part of the Contributor Version) or other devices; or (3) under Patent Claims infringed by Covered Software in the absence of Modifications made by that Contributor.
+3. Distribution Obligations.
+
+3.1. Availability of Source Code.
+Any Covered Software that You distribute or otherwise make available in Executable form must also be made available in Source Code form and that Source Code form must be distributed only under the terms of this License. You must include a copy of this License with every copy of the Source Code form of the Covered Software You distribute or otherwise make available. You must inform recipients of any such Covered Software in Executable form as to how they can obtain such Covered Software in Source Code form in a reasonable manner on or through a medium customarily used for software exchange.
+3.2. Modifications.
+The Modifications that You create or to which You contribute are governed by the terms of this License. You represent that You believe Your Modifications are Your original creation(s) and/or You have sufficient rights to grant the rights conveyed by this License.
+3.3. Required Notices.
+You must include a notice in each of Your Modifications that identifies You as the Contributor of the Modification. You may not remove or alter any copyright, patent or trademark notices contained within the Covered Software, or any notices of licensing or any descriptive text giving attribution to any Contributor or the Initial Developer.
+3.4. Application of Additional Terms.
+You may not offer or impose any terms on any Covered Software in Source Code form that alters or restricts the applicable version of this License or the recipients' rights hereunder. You may choose to offer, and to charge a fee for, warranty, support, indemnity or liability obligations to one or more recipients of Covered Software. However, you may do so only on Your own behalf, and not on behalf of the Initial Developer or any Contributor. You must make it absolutely clear that any such warranty, support, indemnity or liability obligation is offered by You alone, and You hereby agree to indemnify the Initial Developer and every Contributor for any liability incurred by the Initial Developer or such Contributor as a result of warranty, support, indemnity or liability terms You offer.
+3.5. Distribution of Executable Versions.
+You may distribute the Executable form of the Covered Software under the terms of this License or under the terms of a license of Your choice, which may contain terms different from this License, provided that You are in compliance with the terms of this License and that the license for the Executable form does not attempt to limit or alter the recipient's rights in the Source Code form from the rights set forth in this License. If You distribute the Covered Software in Executable form under a different license, You must make it absolutely clear that any terms which differ from this License are offered by You alone, not by the Initial Developer or Contributor. You hereby agree to indemnify the Initial Developer and every Contributor for any liability incurred by the Initial Developer or such Contributor as a result of any such terms You offer.
+3.6. Larger Works.
+You may create a Larger Work by combining Covered Software with other code not governed by the terms of this License and distribute the Larger Work as a single product. In such a case, You must make sure the requirements of this License are fulfilled for the Covered Software.
+4. Versions of the License.
+
+4.1. New Versions.
+Oracle is the initial license steward and may publish revised and/or new versions of this License from time to time. Each version will be given a distinguishing version number. Except as provided in Section 4.3, no one other than the license steward has the right to modify this License.
+4.2. Effect of New Versions.
+You may always continue to use, distribute or otherwise make the Covered Software available under the terms of the version of the License under which You originally received the Covered Software. If the Initial Developer includes a notice in the Original Software prohibiting it from being distributed or otherwise made available under any subsequent version of the License, You must distribute and make the Covered Software available under the terms of the version of the License under which You originally received the Covered Software. Otherwise, You may also choose to use, distribute or otherwise make the Covered Software available under the terms of any subsequent version of the License published by the license steward.
+4.3. Modified Versions.
+When You are an Initial Developer and You want to create a new license for Your Original Software, You may create and use a modified version of this License if You: (a) rename the license and remove any references to the name of the license steward (except to note that the license differs from this License); and (b) otherwise make it clear that the license contains terms which differ from this License.
+5. DISCLAIMER OF WARRANTY.
+
+COVERED SOFTWARE IS PROVIDED UNDER THIS LICENSE ON AN “AS IS” BASIS, WITHOUT WARRANTY OF ANY KIND, EITHER EXPRESSED OR IMPLIED, INCLUDING, WITHOUT LIMITATION, WARRANTIES THAT THE COVERED SOFTWARE IS FREE OF DEFECTS, MERCHANTABLE, FIT FOR A PARTICULAR PURPOSE OR NON-INFRINGING. THE ENTIRE RISK AS TO THE QUALITY AND PERFORMANCE OF THE COVERED SOFTWARE IS WITH YOU. SHOULD ANY COVERED SOFTWARE PROVE DEFECTIVE IN ANY RESPECT, YOU (NOT THE INITIAL DEVELOPER OR ANY OTHER CONTRIBUTOR) ASSUME THE COST OF ANY NECESSARY SERVICING, REPAIR OR CORRECTION. THIS DISCLAIMER OF WARRANTY CONSTITUTES AN ESSENTIAL PART OF THIS LICENSE. NO USE OF ANY COVERED SOFTWARE IS AUTHORIZED HEREUNDER EXCEPT UNDER THIS DISCLAIMER.
+
+6. TERMINATION.
+
+6.1. This License and the rights granted hereunder will terminate automatically if You fail to comply with terms herein and fail to cure such breach within 30 days of becoming aware of the breach. Provisions which, by their nature, must remain in effect beyond the termination of this License shall survive.
+6.2. If You assert a patent infringement claim (excluding declaratory judgment actions) against Initial Developer or a Contributor (the Initial Developer or Contributor against whom You assert such claim is referred to as “Participant”) alleging that the Participant Software (meaning the Contributor Version where the Participant is a Contributor or the Original Software where the Participant is the Initial Developer) directly or indirectly infringes any patent, then any and all rights granted directly or indirectly to You by such Participant, the Initial Developer (if the Initial Developer is not the Participant) and all Contributors under Sections 2.1 and/or 2.2 of this License shall, upon 60 days notice from Participant terminate prospectively and automatically at the expiration of such 60 day notice period, unless if within such 60 day period You withdraw Your claim with respect to the Participant Software against such Participant either unilaterally or pursuant to a written agreement with Participant.
+6.3. If You assert a patent infringement claim against Participant alleging that the Participant Software directly or indirectly infringes any patent where such claim is resolved (such as by license or settlement) prior to the initiation of patent infringement litigation, then the reasonable value of the licenses granted by such Participant under Sections 2.1 or 2.2 shall be taken into account in determining the amount or value of any payment or license.
+6.4. In the event of termination under Sections 6.1 or 6.2 above, all end user licenses that have been validly granted by You or any distributor hereunder prior to termination (excluding licenses granted to You by any distributor) shall survive termination.
+7. LIMITATION OF LIABILITY.
+
+UNDER NO CIRCUMSTANCES AND UNDER NO LEGAL THEORY, WHETHER TORT (INCLUDING NEGLIGENCE), CONTRACT, OR OTHERWISE, SHALL YOU, THE INITIAL DEVELOPER, ANY OTHER CONTRIBUTOR, OR ANY DISTRIBUTOR OF COVERED SOFTWARE, OR ANY SUPPLIER OF ANY OF SUCH PARTIES, BE LIABLE TO ANY PERSON FOR ANY INDIRECT, SPECIAL, INCIDENTAL, OR CONSEQUENTIAL DAMAGES OF ANY CHARACTER INCLUDING, WITHOUT LIMITATION, DAMAGES FOR LOSS OF GOODWILL, WORK STOPPAGE, COMPUTER FAILURE OR MALFUNCTION, OR ANY AND ALL OTHER COMMERCIAL DAMAGES OR LOSSES, EVEN IF SUCH PARTY SHALL HAVE BEEN INFORMED OF THE POSSIBILITY OF SUCH DAMAGES. THIS LIMITATION OF LIABILITY SHALL NOT APPLY TO LIABILITY FOR DEATH OR PERSONAL INJURY RESULTING FROM SUCH PARTY'S NEGLIGENCE TO THE EXTENT APPLICABLE LAW PROHIBITS SUCH LIMITATION. SOME JURISDICTIONS DO NOT ALLOW THE EXCLUSION OR LIMITATION OF INCIDENTAL OR CONSEQUENTIAL DAMAGES, SO THIS EXCLUSION AND LIMITATION MAY NOT APPLY TO YOU.
+
+8. U.S. GOVERNMENT END USERS.
+
+The Covered Software is a “commercial item,” as that term is defined in 48 C.F.R. 2.101 (Oct. 1995), consisting of “commercial computer software” (as that term is defined at 48 C.F.R. § 252.227-7014(a)(1)) and “commercial computer software documentation” as such terms are used in 48 C.F.R. 12.212 (Sept. 1995). Consistent with 48 C.F.R. 12.212 and 48 C.F.R. 227.7202-1 through 227.7202-4 (June 1995), all U.S. Government End Users acquire Covered Software with only those rights set forth herein. This U.S. Government Rights clause is in lieu of, and supersedes, any other FAR, DFAR, or other clause or provision that addresses Government rights in computer software under this License.
+
+9. MISCELLANEOUS.
+
+This License represents the complete agreement concerning subject matter hereof. If any provision of this License is held to be unenforceable, such provision shall be reformed only to the extent necessary to make it enforceable. This License shall be governed by the law of the jurisdiction specified in a notice contained within the Original Software (except to the extent applicable law, if any, provides otherwise), excluding such jurisdiction's conflict-of-law provisions. Any litigation relating to this License shall be subject to the jurisdiction of the courts located in the jurisdiction and venue specified in a notice contained within the Original Software, with the losing party responsible for costs, including, without limitation, court costs and reasonable attorneys' fees and expenses. The application of the United Nations Convention on Contracts for the International Sale of Goods is expressly excluded. Any law or regulation which provides that the language of a contract shall be construed against the drafter shall not apply to this License. You agree that You alone are responsible for compliance with the United States export administration regulations (and the export control laws and regulation of any other countries) when You use, distribute or otherwise make available any Covered Software.
+
+10. RESPONSIBILITY FOR CLAIMS.
+
+As between Initial Developer and the Contributors, each party is responsible for claims and damages arising, directly or indirectly, out of its utilization of rights under this License and You agree to work with Initial Developer and Contributors to distribute such responsibility on an equitable basis. Nothing herein is intended or shall be deemed to constitute any admission of liability.
+
+NOTICE PURSUANT TO SECTION 9 OF THE COMMON DEVELOPMENT AND DISTRIBUTION LICENSE (CDDL)
+
+The code released under the CDDL shall be governed by the laws of the State of California (excluding conflict-of-law provisions). Any litigation relating to this License shall be subject to the jurisdiction of the Federal Courts of the Northern District of California and the state courts of the State of California, with venue lying in Santa Clara County, California.
+
+
+
+The GNU General Public License (GPL) Version 2, June 1991
+
+Copyright (C) 1989, 1991 Free Software Foundation, Inc.
+59 Temple Place, Suite 330, Boston, MA 02111-1307 USA
+
+Everyone is permitted to copy and distribute verbatim copies
+of this license document, but changing it is not allowed.
+
+Preamble
+
+The licenses for most software are designed to take away your freedom to share and change it. By contrast, the GNU General Public License is intended to guarantee your freedom to share and change free software--to make sure the software is free for all its users. This General Public License applies to most of the Free Software Foundation's software and to any other program whose authors commit to using it. (Some other Free Software Foundation software is covered by the GNU Library General Public License instead.) You can apply it to your programs, too.
+
+When we speak of free software, we are referring to freedom, not price. Our General Public Licenses are designed to make sure that you have the freedom to distribute copies of free software (and charge for this service if you wish), that you receive source code or can get it if you want it, that you can change the software or use pieces of it in new free programs; and that you know you can do these things.
+
+To protect your rights, we need to make restrictions that forbid anyone to deny you these rights or to ask you to surrender the rights. These restrictions translate to certain responsibilities for you if you distribute copies of the software, or if you modify it.
+
+For example, if you distribute copies of such a program, whether gratis or for a fee, you must give the recipients all the rights that you have. You must make sure that they, too, receive or can get the source code. And you must show them these terms so they know their rights.
+
+We protect your rights with two steps: (1) copyright the software, and (2) offer you this license which gives you legal permission to copy, distribute and/or modify the software.
+
+Also, for each author's protection and ours, we want to make certain that everyone understands that there is no warranty for this free software. If the software is modified by someone else and passed on, we want its recipients to know that what they have is not the original, so that any problems introduced by others will not reflect on the original authors' reputations.
+
+Finally, any free program is threatened constantly by software patents. We wish to avoid the danger that redistributors of a free program will individually obtain patent licenses, in effect making the program proprietary. To prevent this, we have made it clear that any patent must be licensed for everyone's free use or not licensed at all.
+
+The precise terms and conditions for copying, distribution and modification follow.
+
+TERMS AND CONDITIONS FOR COPYING, DISTRIBUTION AND MODIFICATION
+
+0. This License applies to any program or other work which contains a notice placed by the copyright holder saying it may be distributed under the terms of this General Public License. The "Program", below, refers to any such program or work, and a "work based on the Program" means either the Program or any derivative work under copyright law: that is to say, a work containing the Program or a portion of it, either verbatim or with modifications and/or translated into another language. (Hereinafter, translation is included without limitation in the term "modification".) Each licensee is addressed as "you".
+
+Activities other than copying, distribution and modification are not covered by this License; they are outside its scope. The act of running the Program is not restricted, and the output from the Program is covered only if its contents constitute a work based on the Program (independent of having been made by running the Program). Whether that is true depends on what the Program does.
+
+1. You may copy and distribute verbatim copies of the Program's source code as you receive it, in any medium, provided that you conspicuously and appropriately publish on each copy an appropriate copyright notice and disclaimer of warranty; keep intact all the notices that refer to this License and to the absence of any warranty; and give any other recipients of the Program a copy of this License along with the Program.
+
+You may charge a fee for the physical act of transferring a copy, and you may at your option offer warranty protection in exchange for a fee.
+
+2. You may modify your copy or copies of the Program or any portion of it, thus forming a work based on the Program, and copy and distribute such modifications or work under the terms of Section 1 above, provided that you also meet all of these conditions:
+
+a) You must cause the modified files to carry prominent notices stating that you changed the files and the date of any change.
+b) You must cause any work that you distribute or publish, that in whole or in part contains or is derived from the Program or any part thereof, to be licensed as a whole at no charge to all third parties under the terms of this License.
+c) If the modified program normally reads commands interactively when run, you must cause it, when started running for such interactive use in the most ordinary way, to print or display an announcement including an appropriate copyright notice and a notice that there is no warranty (or else, saying that you provide a warranty) and that users may redistribute the program under these conditions, and telling the user how to view a copy of this License. (Exception: if the Program itself is interactive but does not normally print such an announcement, your work based on the Program is not required to print an announcement.)
+These requirements apply to the modified work as a whole. If identifiable sections of that work are not derived from the Program, and can be reasonably considered independent and separate works in themselves, then this License, and its terms, do not apply to those sections when you distribute them as separate works. But when you distribute the same sections as part of a whole which is a work based on the Program, the distribution of the whole must be on the terms of this License, whose permissions for other licensees extend to the entire whole, and thus to each and every part regardless of who wrote it.
+
+Thus, it is not the intent of this section to claim rights or contest your rights to work written entirely by you; rather, the intent is to exercise the right to control the distribution of derivative or collective works based on the Program.
+
+In addition, mere aggregation of another work not based on the Program with the Program (or with a work based on the Program) on a volume of a storage or distribution medium does not bring the other work under the scope of this License.
+
+3. You may copy and distribute the Program (or a work based on it, under Section 2) in object code or executable form under the terms of Sections 1 and 2 above provided that you also do one of the following:
+
+a) Accompany it with the complete corresponding machine-readable source code, which must be distributed under the terms of Sections 1 and 2 above on a medium customarily used for software interchange; or,
+b) Accompany it with a written offer, valid for at least three years, to give any third party, for a charge no more than your cost of physically performing source distribution, a complete machine-readable copy of the corresponding source code, to be distributed under the terms of Sections 1 and 2 above on a medium customarily used for software interchange; or,
+c) Accompany it with the information you received as to the offer to distribute corresponding source code. (This alternative is allowed only for noncommercial distribution and only if you received the program in object code or executable form with such an offer, in accord with Subsection b above.)
+The source code for a work means the preferred form of the work for making modifications to it. For an executable work, complete source code means all the source code for all modules it contains, plus any associated interface definition files, plus the scripts used to control compilation and installation of the executable. However, as a special exception, the source code distributed need not include anything that is normally distributed (in either source or binary form) with the major components (compiler, kernel, and so on) of the operating system on which the executable runs, unless that component itself accompanies the executable.
+
+If distribution of executable or object code is made by offering access to copy from a designated place, then offering equivalent access to copy the source code from the same place counts as distribution of the source code, even though third parties are not compelled to copy the source along with the object code.
+
+4. You may not copy, modify, sublicense, or distribute the Program except as expressly provided under this License. Any attempt otherwise to copy, modify, sublicense or distribute the Program is void, and will automatically terminate your rights under this License. However, parties who have received copies, or rights, from you under this License will not have their licenses terminated so long as such parties remain in full compliance.
+
+5. You are not required to accept this License, since you have not signed it. However, nothing else grants you permission to modify or distribute the Program or its derivative works. These actions are prohibited by law if you do not accept this License. Therefore, by modifying or distributing the Program (or any work based on the Program), you indicate your acceptance of this License to do so, and all its terms and conditions for copying, distributing or modifying the Program or works based on it.
+
+6. Each time you redistribute the Program (or any work based on the Program), the recipient automatically receives a license from the original licensor to copy, distribute or modify the Program subject to these terms and conditions. You may not impose any further restrictions on the recipients' exercise of the rights granted herein. You are not responsible for enforcing compliance by third parties to this License.
+
+7. If, as a consequence of a court judgment or allegation of patent infringement or for any other reason (not limited to patent issues), conditions are imposed on you (whether by court order, agreement or otherwise) that contradict the conditions of this License, they do not excuse you from the conditions of this License. If you cannot distribute so as to satisfy simultaneously your obligations under this License and any other pertinent obligations, then as a consequence you may not distribute the Program at all. For example, if a patent license would not permit royalty-free redistribution of the Program by all those who receive copies directly or indirectly through you, then the only way you could satisfy both it and this License would be to refrain entirely from distribution of the Program.
+
+If any portion of this section is held invalid or unenforceable under any particular circumstance, the balance of the section is intended to apply and the section as a whole is intended to apply in other circumstances.
+
+It is not the purpose of this section to induce you to infringe any patents or other property right claims or to contest validity of any such claims; this section has the sole purpose of protecting the integrity of the free software distribution system, which is implemented by public license practices. Many people have made generous contributions to the wide range of software distributed through that system in reliance on consistent application of that system; it is up to the author/donor to decide if he or she is willing to distribute software through any other system and a licensee cannot impose that choice.
+
+This section is intended to make thoroughly clear what is believed to be a consequence of the rest of this License.
+
+8. If the distribution and/or use of the Program is restricted in certain countries either by patents or by copyrighted interfaces, the original copyright holder who places the Program under this License may add an explicit geographical distribution limitation excluding those countries, so that distribution is permitted only in or among countries not thus excluded. In such case, this License incorporates the limitation as if written in the body of this License.
+
+9. The Free Software Foundation may publish revised and/or new versions of the General Public License from time to time. Such new versions will be similar in spirit to the present version, but may differ in detail to address new problems or concerns.
+
+Each version is given a distinguishing version number. If the Program specifies a version number of this License which applies to it and "any later version", you have the option of following the terms and conditions either of that version or of any later version published by the Free Software Foundation. If the Program does not specify a version number of this License, you may choose any version ever published by the Free Software Foundation.
+
+10. If you wish to incorporate parts of the Program into other free programs whose distribution conditions are different, write to the author to ask for permission. For software which is copyrighted by the Free Software Foundation, write to the Free Software Foundation; we sometimes make exceptions for this. Our decision will be guided by the two goals of preserving the free status of all derivatives of our free software and of promoting the sharing and reuse of software generally.
+
+NO WARRANTY
+
+11. BECAUSE THE PROGRAM IS LICENSED FREE OF CHARGE, THERE IS NO WARRANTY FOR THE PROGRAM, TO THE EXTENT PERMITTED BY APPLICABLE LAW. EXCEPT WHEN OTHERWISE STATED IN WRITING THE COPYRIGHT HOLDERS AND/OR OTHER PARTIES PROVIDE THE PROGRAM "AS IS" WITHOUT WARRANTY OF ANY KIND, EITHER EXPRESSED OR IMPLIED, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE. THE ENTIRE RISK AS TO THE QUALITY AND PERFORMANCE OF THE PROGRAM IS WITH YOU. SHOULD THE PROGRAM PROVE DEFECTIVE, YOU ASSUME THE COST OF ALL NECESSARY SERVICING, REPAIR OR CORRECTION.
+
+12. IN NO EVENT UNLESS REQUIRED BY APPLICABLE LAW OR AGREED TO IN WRITING WILL ANY COPYRIGHT HOLDER, OR ANY OTHER PARTY WHO MAY MODIFY AND/OR REDISTRIBUTE THE PROGRAM AS PERMITTED ABOVE, BE LIABLE TO YOU FOR DAMAGES, INCLUDING ANY GENERAL, SPECIAL, INCIDENTAL OR CONSEQUENTIAL DAMAGES ARISING OUT OF THE USE OR INABILITY TO USE THE PROGRAM (INCLUDING BUT NOT LIMITED TO LOSS OF DATA OR DATA BEING RENDERED INACCURATE OR LOSSES SUSTAINED BY YOU OR THIRD PARTIES OR A FAILURE OF THE PROGRAM TO OPERATE WITH ANY OTHER PROGRAMS), EVEN IF SUCH HOLDER OR OTHER PARTY HAS BEEN ADVISED OF THE POSSIBILITY OF SUCH DAMAGES.
+
+END OF TERMS AND CONDITIONS
+
+How to Apply These Terms to Your New Programs
+
+If you develop a new program, and you want it to be of the greatest possible use to the public, the best way to achieve this is to make it free software which everyone can redistribute and change under these terms.
+
+To do so, attach the following notices to the program. It is safest to attach them to the start of each source file to most effectively convey the exclusion of warranty; and each file should have at least the "copyright" line and a pointer to where the full notice is found.
+
+One line to give the program's name and a brief idea of what it does.
+Copyright (C) <year> <name of author>
+This program is free software; you can redistribute it and/or modify it under the terms of the GNU General Public License as published by the Free Software Foundation; either version 2 of the License, or (at your option) any later version.
+This program is distributed in the hope that it will be useful, but WITHOUT ANY WARRANTY; without even the implied warranty of MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the GNU General Public License for more details.
+You should have received a copy of the GNU General Public License along with this program; if not, write to the Free Software Foundation, Inc., 59 Temple Place, Suite 330, Boston, MA 02111-1307 USA
+Also add information on how to contact you by electronic and paper mail.
+
+If the program is interactive, make it output a short notice like this when it starts in an interactive mode:
+
+Gnomovision version 69, Copyright (C) year name of author Gnomovision comes with ABSOLUTELY NO WARRANTY; for details type `show w'. This is free software, and you are welcome to redistribute it under certain conditions; type `show c' for details.
+The hypothetical commands `show w' and `show c' should show the appropriate parts of the General Public License. Of course, the commands you use may be called something other than `show w' and `show c'; they could even be mouse-clicks or menu items--whatever suits your program.
+
+You should also get your employer (if you work as a programmer) or your school, if any, to sign a "copyright disclaimer" for the program, if necessary. Here is a sample; alter the names:
+
+Yoyodyne, Inc., hereby disclaims all copyright interest in the program `Gnomovision' (which makes passes at compilers) written by James Hacker.
+signature of Ty Coon, 1 April 1989
+Ty Coon, President of Vice
+This General Public License does not permit incorporating your program into proprietary programs. If your program is a subroutine library, you may consider it more useful to permit linking proprietary applications with the library. If this is what you want to do, use the GNU Library General Public License instead of this License.
+
+"CLASSPATH" EXCEPTION TO THE GPL VERSION 2
+
+Certain source files distributed by Oracle are subject to the following clarification and special exception to the GPL Version 2, but only where Oracle has expressly included in the particular source file's header the words "Oracle designates this particular file as subject to the "Classpath" exception as provided by Oracle in the License file that accompanied this code." 
+
+Linking this library statically or dynamically with other modules is making a combined work based on this library.  Thus, the terms and conditions of the GNU General Public License Version 2 cover the whole combination. 
+
+As a special exception, the copyright holders of this library give you permission to link this library with independent modules to produce an executable, regardless of the license terms of these independent modules, and to copy and distribute the resulting executable under terms of your choice, provided that you also meet, for each linked independent module, the terms and conditions of the license of that module.  An independent module is a module which is not derived from or based on this library.  If you modify this library, you may extend this exception to your version of the library, but you are not obligated to do so.  If you do not wish to do so, delete this exception statement from your version.
+
+
+
+
diff --git a/licenses/LICENSE-jcodings.txt b/licenses/LICENSE-jcodings.txt
new file mode 100644
index 0000000..f803194
--- /dev/null
+++ b/licenses/LICENSE-jcodings.txt
@@ -0,0 +1,18 @@
+Permission is hereby granted, free of charge, to any person obtaining a copy of
+this software and associated documentation files (the "Software"), to deal in
+the Software without restriction, including without limitation the rights to
+use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies
+of the Software, and to permit persons to whom the Software is furnished to do
+so, subject to the following conditions:
+
+The above copyright notice and this permission notice shall be included in all
+copies or substantial portions of the Software.
+
+THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
+SOFTWARE.
+
diff --git a/licenses/LICENSE-jedis.txt b/licenses/LICENSE-jedis.txt
new file mode 100644
index 0000000..11ed4ba
--- /dev/null
+++ b/licenses/LICENSE-jedis.txt
@@ -0,0 +1,22 @@
+Copyright (c) 2010 Jonathan Leibiusky
+
+Permission is hereby granted, free of charge, to any person
+obtaining a copy of this software and associated documentation
+files (the "Software"), to deal in the Software without
+restriction, including without limitation the rights to use,
+copy, modify, merge, publish, distribute, sublicense, and/or sell
+copies of the Software, and to permit persons to whom the
+Software is furnished to do so, subject to the following
+conditions:
+
+The above copyright notice and this permission notice shall be
+included in all copies or substantial portions of the Software.
+
+THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND,
+EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES
+OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND
+NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT
+HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY,
+WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING
+FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR
+OTHER DEALINGS IN THE SOFTWARE.
diff --git a/licenses/LICENSE-jersey-guice.txt b/licenses/LICENSE-jersey-guice.txt
new file mode 100644
index 0000000..1507ec7
--- /dev/null
+++ b/licenses/LICENSE-jersey-guice.txt
@@ -0,0 +1,210 @@
+COMMON DEVELOPMENT AND DISTRIBUTION LICENSE (CDDL)Version 1.1
+
+1. Definitions.
+
+1.1. “Contributor” means each individual or entity that creates or contributes to the creation of Modifications.
+1.2. “Contributor Version” means the combination of the Original Software, prior Modifications used by a Contributor (if any), and the Modifications made by that particular Contributor.
+1.3. “Covered Software” means (a) the Original Software, or (b) Modifications, or (c) the combination of files containing Original Software with files containing Modifications, in each case including portions thereof.
+1.4. “Executable” means the Covered Software in any form other than Source Code.
+1.5. “Initial Developer” means the individual or entity that first makes Original Software available under this License.
+1.6. “Larger Work” means a work which combines Covered Software or portions thereof with code not governed by the terms of this License.
+1.7. “License” means this document.
+1.8. “Licensable” means having the right to grant, to the maximum extent possible, whether at the time of the initial grant or subsequently acquired, any and all of the rights conveyed herein.
+1.9. “Modifications” means the Source Code and Executable form of any of the following:
+A. Any file that results from an addition to, deletion from or modification of the contents of a file containing Original Software or previous Modifications;
+B. Any new file that contains any part of the Original Software or previous Modification; or
+C. Any new file that is contributed or otherwise made available under the terms of this License.
+1.10. “Original Software” means the Source Code and Executable form of computer software code that is originally released under this License.
+1.11. “Patent Claims” means any patent claim(s), now owned or hereafter acquired, including without limitation, method, process, and apparatus claims, in any patent Licensable by grantor.
+1.12. “Source Code” means (a) the common form of computer software code in which modifications are made and (b) associated documentation included in or with such code.
+1.13. “You” (or “Your”) means an individual or a legal entity exercising rights under, and complying with all of the terms of, this License. For legal entities, “You” includes any entity which controls, is controlled by, or is under common control with You. For purposes of this definition, “control” means (a) the power, direct or indirect, to cause the direction or management of such entity, whether by contract or otherwise, or (b) ownership of more than fifty percent (50%) of the outstanding shares or beneficial ownership of such entity.
+2. License Grants.
+
+2.1. The Initial Developer Grant.
+Conditioned upon Your compliance with Section 3.1 below and subject to third party intellectual property claims, the Initial Developer hereby grants You a world-wide, royalty-free, non-exclusive license:
+(a) under intellectual property rights (other than patent or trademark) Licensable by Initial Developer, to use, reproduce, modify, display, perform, sublicense and distribute the Original Software (or portions thereof), with or without Modifications, and/or as part of a Larger Work; and
+(b) under Patent Claims infringed by the making, using or selling of Original Software, to make, have made, use, practice, sell, and offer for sale, and/or otherwise dispose of the Original Software (or portions thereof).
+(c) The licenses granted in Sections 2.1(a) and (b) are effective on the date Initial Developer first distributes or otherwise makes the Original Software available to a third party under the terms of this License.
+(d) Notwithstanding Section 2.1(b) above, no patent license is granted: (1) for code that You delete from the Original Software, or (2) for infringements caused by: (i) the modification of the Original Software, or (ii) the combination of the Original Software with other software or devices.
+2.2. Contributor Grant.
+Conditioned upon Your compliance with Section 3.1 below and subject to third party intellectual property claims, each Contributor hereby grants You a world-wide, royalty-free, non-exclusive license:
+(a) under intellectual property rights (other than patent or trademark) Licensable by Contributor to use, reproduce, modify, display, perform, sublicense and distribute the Modifications created by such Contributor (or portions thereof), either on an unmodified basis, with other Modifications, as Covered Software and/or as part of a Larger Work; and
+(b) under Patent Claims infringed by the making, using, or selling of Modifications made by that Contributor either alone and/or in combination with its Contributor Version (or portions of such combination), to make, use, sell, offer for sale, have made, and/or otherwise dispose of: (1) Modifications made by that Contributor (or portions thereof); and (2) the combination of Modifications made by that Contributor with its Contributor Version (or portions of such combination).
+(c) The licenses granted in Sections 2.2(a) and 2.2(b) are effective on the date Contributor first distributes or otherwise makes the Modifications available to a third party.
+(d) Notwithstanding Section 2.2(b) above, no patent license is granted: (1) for any code that Contributor has deleted from the Contributor Version; (2) for infringements caused by: (i) third party modifications of Contributor Version, or (ii) the combination of Modifications made by that Contributor with other software (except as part of the Contributor Version) or other devices; or (3) under Patent Claims infringed by Covered Software in the absence of Modifications made by that Contributor.
+3. Distribution Obligations.
+
+3.1. Availability of Source Code.
+Any Covered Software that You distribute or otherwise make available in Executable form must also be made available in Source Code form and that Source Code form must be distributed only under the terms of this License. You must include a copy of this License with every copy of the Source Code form of the Covered Software You distribute or otherwise make available. You must inform recipients of any such Covered Software in Executable form as to how they can obtain such Covered Software in Source Code form in a reasonable manner on or through a medium customarily used for software exchange.
+3.2. Modifications.
+The Modifications that You create or to which You contribute are governed by the terms of this License. You represent that You believe Your Modifications are Your original creation(s) and/or You have sufficient rights to grant the rights conveyed by this License.
+3.3. Required Notices.
+You must include a notice in each of Your Modifications that identifies You as the Contributor of the Modification. You may not remove or alter any copyright, patent or trademark notices contained within the Covered Software, or any notices of licensing or any descriptive text giving attribution to any Contributor or the Initial Developer.
+3.4. Application of Additional Terms.
+You may not offer or impose any terms on any Covered Software in Source Code form that alters or restricts the applicable version of this License or the recipients' rights hereunder. You may choose to offer, and to charge a fee for, warranty, support, indemnity or liability obligations to one or more recipients of Covered Software. However, you may do so only on Your own behalf, and not on behalf of the Initial Developer or any Contributor. You must make it absolutely clear that any such warranty, support, indemnity or liability obligation is offered by You alone, and You hereby agree to indemnify the Initial Developer and every Contributor for any liability incurred by the Initial Developer or such Contributor as a result of warranty, support, indemnity or liability terms You offer.
+3.5. Distribution of Executable Versions.
+You may distribute the Executable form of the Covered Software under the terms of this License or under the terms of a license of Your choice, which may contain terms different from this License, provided that You are in compliance with the terms of this License and that the license for the Executable form does not attempt to limit or alter the recipient's rights in the Source Code form from the rights set forth in this License. If You distribute the Covered Software in Executable form under a different license, You must make it absolutely clear that any terms which differ from this License are offered by You alone, not by the Initial Developer or Contributor. You hereby agree to indemnify the Initial Developer and every Contributor for any liability incurred by the Initial Developer or such Contributor as a result of any such terms You offer.
+3.6. Larger Works.
+You may create a Larger Work by combining Covered Software with other code not governed by the terms of this License and distribute the Larger Work as a single product. In such a case, You must make sure the requirements of this License are fulfilled for the Covered Software.
+4. Versions of the License.
+
+4.1. New Versions.
+Oracle is the initial license steward and may publish revised and/or new versions of this License from time to time. Each version will be given a distinguishing version number. Except as provided in Section 4.3, no one other than the license steward has the right to modify this License.
+4.2. Effect of New Versions.
+You may always continue to use, distribute or otherwise make the Covered Software available under the terms of the version of the License under which You originally received the Covered Software. If the Initial Developer includes a notice in the Original Software prohibiting it from being distributed or otherwise made available under any subsequent version of the License, You must distribute and make the Covered Software available under the terms of the version of the License under which You originally received the Covered Software. Otherwise, You may also choose to use, distribute or otherwise make the Covered Software available under the terms of any subsequent version of the License published by the license steward.
+4.3. Modified Versions.
+When You are an Initial Developer and You want to create a new license for Your Original Software, You may create and use a modified version of this License if You: (a) rename the license and remove any references to the name of the license steward (except to note that the license differs from this License); and (b) otherwise make it clear that the license contains terms which differ from this License.
+5. DISCLAIMER OF WARRANTY.
+
+COVERED SOFTWARE IS PROVIDED UNDER THIS LICENSE ON AN “AS IS” BASIS, WITHOUT WARRANTY OF ANY KIND, EITHER EXPRESSED OR IMPLIED, INCLUDING, WITHOUT LIMITATION, WARRANTIES THAT THE COVERED SOFTWARE IS FREE OF DEFECTS, MERCHANTABLE, FIT FOR A PARTICULAR PURPOSE OR NON-INFRINGING. THE ENTIRE RISK AS TO THE QUALITY AND PERFORMANCE OF THE COVERED SOFTWARE IS WITH YOU. SHOULD ANY COVERED SOFTWARE PROVE DEFECTIVE IN ANY RESPECT, YOU (NOT THE INITIAL DEVELOPER OR ANY OTHER CONTRIBUTOR) ASSUME THE COST OF ANY NECESSARY SERVICING, REPAIR OR CORRECTION. THIS DISCLAIMER OF WARRANTY CONSTITUTES AN ESSENTIAL PART OF THIS LICENSE. NO USE OF ANY COVERED SOFTWARE IS AUTHORIZED HEREUNDER EXCEPT UNDER THIS DISCLAIMER.
+
+6. TERMINATION.
+
+6.1. This License and the rights granted hereunder will terminate automatically if You fail to comply with terms herein and fail to cure such breach within 30 days of becoming aware of the breach. Provisions which, by their nature, must remain in effect beyond the termination of this License shall survive.
+6.2. If You assert a patent infringement claim (excluding declaratory judgment actions) against Initial Developer or a Contributor (the Initial Developer or Contributor against whom You assert such claim is referred to as “Participant”) alleging that the Participant Software (meaning the Contributor Version where the Participant is a Contributor or the Original Software where the Participant is the Initial Developer) directly or indirectly infringes any patent, then any and all rights granted directly or indirectly to You by such Participant, the Initial Developer (if the Initial Developer is not the Participant) and all Contributors under Sections 2.1 and/or 2.2 of this License shall, upon 60 days notice from Participant terminate prospectively and automatically at the expiration of such 60 day notice period, unless if within such 60 day period You withdraw Your claim with respect to the Participant Software against such Participant either unilaterally or pursuant to a written agreement with Participant.
+6.3. If You assert a patent infringement claim against Participant alleging that the Participant Software directly or indirectly infringes any patent where such claim is resolved (such as by license or settlement) prior to the initiation of patent infringement litigation, then the reasonable value of the licenses granted by such Participant under Sections 2.1 or 2.2 shall be taken into account in determining the amount or value of any payment or license.
+6.4. In the event of termination under Sections 6.1 or 6.2 above, all end user licenses that have been validly granted by You or any distributor hereunder prior to termination (excluding licenses granted to You by any distributor) shall survive termination.
+7. LIMITATION OF LIABILITY.
+
+UNDER NO CIRCUMSTANCES AND UNDER NO LEGAL THEORY, WHETHER TORT (INCLUDING NEGLIGENCE), CONTRACT, OR OTHERWISE, SHALL YOU, THE INITIAL DEVELOPER, ANY OTHER CONTRIBUTOR, OR ANY DISTRIBUTOR OF COVERED SOFTWARE, OR ANY SUPPLIER OF ANY OF SUCH PARTIES, BE LIABLE TO ANY PERSON FOR ANY INDIRECT, SPECIAL, INCIDENTAL, OR CONSEQUENTIAL DAMAGES OF ANY CHARACTER INCLUDING, WITHOUT LIMITATION, DAMAGES FOR LOSS OF GOODWILL, WORK STOPPAGE, COMPUTER FAILURE OR MALFUNCTION, OR ANY AND ALL OTHER COMMERCIAL DAMAGES OR LOSSES, EVEN IF SUCH PARTY SHALL HAVE BEEN INFORMED OF THE POSSIBILITY OF SUCH DAMAGES. THIS LIMITATION OF LIABILITY SHALL NOT APPLY TO LIABILITY FOR DEATH OR PERSONAL INJURY RESULTING FROM SUCH PARTY'S NEGLIGENCE TO THE EXTENT APPLICABLE LAW PROHIBITS SUCH LIMITATION. SOME JURISDICTIONS DO NOT ALLOW THE EXCLUSION OR LIMITATION OF INCIDENTAL OR CONSEQUENTIAL DAMAGES, SO THIS EXCLUSION AND LIMITATION MAY NOT APPLY TO YOU.
+
+8. U.S. GOVERNMENT END USERS.
+
+The Covered Software is a “commercial item,” as that term is defined in 48 C.F.R. 2.101 (Oct. 1995), consisting of “commercial computer software” (as that term is defined at 48 C.F.R. § 252.227-7014(a)(1)) and “commercial computer software documentation” as such terms are used in 48 C.F.R. 12.212 (Sept. 1995). Consistent with 48 C.F.R. 12.212 and 48 C.F.R. 227.7202-1 through 227.7202-4 (June 1995), all U.S. Government End Users acquire Covered Software with only those rights set forth herein. This U.S. Government Rights clause is in lieu of, and supersedes, any other FAR, DFAR, or other clause or provision that addresses Government rights in computer software under this License.
+
+9. MISCELLANEOUS.
+
+This License represents the complete agreement concerning subject matter hereof. If any provision of this License is held to be unenforceable, such provision shall be reformed only to the extent necessary to make it enforceable. This License shall be governed by the law of the jurisdiction specified in a notice contained within the Original Software (except to the extent applicable law, if any, provides otherwise), excluding such jurisdiction's conflict-of-law provisions. Any litigation relating to this License shall be subject to the jurisdiction of the courts located in the jurisdiction and venue specified in a notice contained within the Original Software, with the losing party responsible for costs, including, without limitation, court costs and reasonable attorneys' fees and expenses. The application of the United Nations Convention on Contracts for the International Sale of Goods is expressly excluded. Any law or regulation which provides that the language of a contract shall be construed against the drafter shall not apply to this License. You agree that You alone are responsible for compliance with the United States export administration regulations (and the export control laws and regulation of any other countries) when You use, distribute or otherwise make available any Covered Software.
+
+10. RESPONSIBILITY FOR CLAIMS.
+
+As between Initial Developer and the Contributors, each party is responsible for claims and damages arising, directly or indirectly, out of its utilization of rights under this License and You agree to work with Initial Developer and Contributors to distribute such responsibility on an equitable basis. Nothing herein is intended or shall be deemed to constitute any admission of liability.
+
+NOTICE PURSUANT TO SECTION 9 OF THE COMMON DEVELOPMENT AND DISTRIBUTION LICENSE (CDDL)
+
+The code released under the CDDL shall be governed by the laws of the State of California (excluding conflict-of-law provisions). Any litigation relating to this License shall be subject to the jurisdiction of the Federal Courts of the Northern District of California and the state courts of the State of California, with venue lying in Santa Clara County, California.
+
+
+
+The GNU General Public License (GPL) Version 2, June 1991
+
+Copyright (C) 1989, 1991 Free Software Foundation, Inc.
+59 Temple Place, Suite 330, Boston, MA 02111-1307 USA
+
+Everyone is permitted to copy and distribute verbatim copies
+of this license document, but changing it is not allowed.
+
+Preamble
+
+The licenses for most software are designed to take away your freedom to share and change it. By contrast, the GNU General Public License is intended to guarantee your freedom to share and change free software--to make sure the software is free for all its users. This General Public License applies to most of the Free Software Foundation's software and to any other program whose authors commit to using it. (Some other Free Software Foundation software is covered by the GNU Library General Public License instead.) You can apply it to your programs, too.
+
+When we speak of free software, we are referring to freedom, not price. Our General Public Licenses are designed to make sure that you have the freedom to distribute copies of free software (and charge for this service if you wish), that you receive source code or can get it if you want it, that you can change the software or use pieces of it in new free programs; and that you know you can do these things.
+
+To protect your rights, we need to make restrictions that forbid anyone to deny you these rights or to ask you to surrender the rights. These restrictions translate to certain responsibilities for you if you distribute copies of the software, or if you modify it.
+
+For example, if you distribute copies of such a program, whether gratis or for a fee, you must give the recipients all the rights that you have. You must make sure that they, too, receive or can get the source code. And you must show them these terms so they know their rights.
+
+We protect your rights with two steps: (1) copyright the software, and (2) offer you this license which gives you legal permission to copy, distribute and/or modify the software.
+
+Also, for each author's protection and ours, we want to make certain that everyone understands that there is no warranty for this free software. If the software is modified by someone else and passed on, we want its recipients to know that what they have is not the original, so that any problems introduced by others will not reflect on the original authors' reputations.
+
+Finally, any free program is threatened constantly by software patents. We wish to avoid the danger that redistributors of a free program will individually obtain patent licenses, in effect making the program proprietary. To prevent this, we have made it clear that any patent must be licensed for everyone's free use or not licensed at all.
+
+The precise terms and conditions for copying, distribution and modification follow.
+
+TERMS AND CONDITIONS FOR COPYING, DISTRIBUTION AND MODIFICATION
+
+0. This License applies to any program or other work which contains a notice placed by the copyright holder saying it may be distributed under the terms of this General Public License. The "Program", below, refers to any such program or work, and a "work based on the Program" means either the Program or any derivative work under copyright law: that is to say, a work containing the Program or a portion of it, either verbatim or with modifications and/or translated into another language. (Hereinafter, translation is included without limitation in the term "modification".) Each licensee is addressed as "you".
+
+Activities other than copying, distribution and modification are not covered by this License; they are outside its scope. The act of running the Program is not restricted, and the output from the Program is covered only if its contents constitute a work based on the Program (independent of having been made by running the Program). Whether that is true depends on what the Program does.
+
+1. You may copy and distribute verbatim copies of the Program's source code as you receive it, in any medium, provided that you conspicuously and appropriately publish on each copy an appropriate copyright notice and disclaimer of warranty; keep intact all the notices that refer to this License and to the absence of any warranty; and give any other recipients of the Program a copy of this License along with the Program.
+
+You may charge a fee for the physical act of transferring a copy, and you may at your option offer warranty protection in exchange for a fee.
+
+2. You may modify your copy or copies of the Program or any portion of it, thus forming a work based on the Program, and copy and distribute such modifications or work under the terms of Section 1 above, provided that you also meet all of these conditions:
+
+a) You must cause the modified files to carry prominent notices stating that you changed the files and the date of any change.
+b) You must cause any work that you distribute or publish, that in whole or in part contains or is derived from the Program or any part thereof, to be licensed as a whole at no charge to all third parties under the terms of this License.
+c) If the modified program normally reads commands interactively when run, you must cause it, when started running for such interactive use in the most ordinary way, to print or display an announcement including an appropriate copyright notice and a notice that there is no warranty (or else, saying that you provide a warranty) and that users may redistribute the program under these conditions, and telling the user how to view a copy of this License. (Exception: if the Program itself is interactive but does not normally print such an announcement, your work based on the Program is not required to print an announcement.)
+These requirements apply to the modified work as a whole. If identifiable sections of that work are not derived from the Program, and can be reasonably considered independent and separate works in themselves, then this License, and its terms, do not apply to those sections when you distribute them as separate works. But when you distribute the same sections as part of a whole which is a work based on the Program, the distribution of the whole must be on the terms of this License, whose permissions for other licensees extend to the entire whole, and thus to each and every part regardless of who wrote it.
+
+Thus, it is not the intent of this section to claim rights or contest your rights to work written entirely by you; rather, the intent is to exercise the right to control the distribution of derivative or collective works based on the Program.
+
+In addition, mere aggregation of another work not based on the Program with the Program (or with a work based on the Program) on a volume of a storage or distribution medium does not bring the other work under the scope of this License.
+
+3. You may copy and distribute the Program (or a work based on it, under Section 2) in object code or executable form under the terms of Sections 1 and 2 above provided that you also do one of the following:
+
+a) Accompany it with the complete corresponding machine-readable source code, which must be distributed under the terms of Sections 1 and 2 above on a medium customarily used for software interchange; or,
+b) Accompany it with a written offer, valid for at least three years, to give any third party, for a charge no more than your cost of physically performing source distribution, a complete machine-readable copy of the corresponding source code, to be distributed under the terms of Sections 1 and 2 above on a medium customarily used for software interchange; or,
+c) Accompany it with the information you received as to the offer to distribute corresponding source code. (This alternative is allowed only for noncommercial distribution and only if you received the program in object code or executable form with such an offer, in accord with Subsection b above.)
+The source code for a work means the preferred form of the work for making modifications to it. For an executable work, complete source code means all the source code for all modules it contains, plus any associated interface definition files, plus the scripts used to control compilation and installation of the executable. However, as a special exception, the source code distributed need not include anything that is normally distributed (in either source or binary form) with the major components (compiler, kernel, and so on) of the operating system on which the executable runs, unless that component itself accompanies the executable.
+
+If distribution of executable or object code is made by offering access to copy from a designated place, then offering equivalent access to copy the source code from the same place counts as distribution of the source code, even though third parties are not compelled to copy the source along with the object code.
+
+4. You may not copy, modify, sublicense, or distribute the Program except as expressly provided under this License. Any attempt otherwise to copy, modify, sublicense or distribute the Program is void, and will automatically terminate your rights under this License. However, parties who have received copies, or rights, from you under this License will not have their licenses terminated so long as such parties remain in full compliance.
+
+5. You are not required to accept this License, since you have not signed it. However, nothing else grants you permission to modify or distribute the Program or its derivative works. These actions are prohibited by law if you do not accept this License. Therefore, by modifying or distributing the Program (or any work based on the Program), you indicate your acceptance of this License to do so, and all its terms and conditions for copying, distributing or modifying the Program or works based on it.
+
+6. Each time you redistribute the Program (or any work based on the Program), the recipient automatically receives a license from the original licensor to copy, distribute or modify the Program subject to these terms and conditions. You may not impose any further restrictions on the recipients' exercise of the rights granted herein. You are not responsible for enforcing compliance by third parties to this License.
+
+7. If, as a consequence of a court judgment or allegation of patent infringement or for any other reason (not limited to patent issues), conditions are imposed on you (whether by court order, agreement or otherwise) that contradict the conditions of this License, they do not excuse you from the conditions of this License. If you cannot distribute so as to satisfy simultaneously your obligations under this License and any other pertinent obligations, then as a consequence you may not distribute the Program at all. For example, if a patent license would not permit royalty-free redistribution of the Program by all those who receive copies directly or indirectly through you, then the only way you could satisfy both it and this License would be to refrain entirely from distribution of the Program.
+
+If any portion of this section is held invalid or unenforceable under any particular circumstance, the balance of the section is intended to apply and the section as a whole is intended to apply in other circumstances.
+
+It is not the purpose of this section to induce you to infringe any patents or other property right claims or to contest validity of any such claims; this section has the sole purpose of protecting the integrity of the free software distribution system, which is implemented by public license practices. Many people have made generous contributions to the wide range of software distributed through that system in reliance on consistent application of that system; it is up to the author/donor to decide if he or she is willing to distribute software through any other system and a licensee cannot impose that choice.
+
+This section is intended to make thoroughly clear what is believed to be a consequence of the rest of this License.
+
+8. If the distribution and/or use of the Program is restricted in certain countries either by patents or by copyrighted interfaces, the original copyright holder who places the Program under this License may add an explicit geographical distribution limitation excluding those countries, so that distribution is permitted only in or among countries not thus excluded. In such case, this License incorporates the limitation as if written in the body of this License.
+
+9. The Free Software Foundation may publish revised and/or new versions of the General Public License from time to time. Such new versions will be similar in spirit to the present version, but may differ in detail to address new problems or concerns.
+
+Each version is given a distinguishing version number. If the Program specifies a version number of this License which applies to it and "any later version", you have the option of following the terms and conditions either of that version or of any later version published by the Free Software Foundation. If the Program does not specify a version number of this License, you may choose any version ever published by the Free Software Foundation.
+
+10. If you wish to incorporate parts of the Program into other free programs whose distribution conditions are different, write to the author to ask for permission. For software which is copyrighted by the Free Software Foundation, write to the Free Software Foundation; we sometimes make exceptions for this. Our decision will be guided by the two goals of preserving the free status of all derivatives of our free software and of promoting the sharing and reuse of software generally.
+
+NO WARRANTY
+
+11. BECAUSE THE PROGRAM IS LICENSED FREE OF CHARGE, THERE IS NO WARRANTY FOR THE PROGRAM, TO THE EXTENT PERMITTED BY APPLICABLE LAW. EXCEPT WHEN OTHERWISE STATED IN WRITING THE COPYRIGHT HOLDERS AND/OR OTHER PARTIES PROVIDE THE PROGRAM "AS IS" WITHOUT WARRANTY OF ANY KIND, EITHER EXPRESSED OR IMPLIED, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE. THE ENTIRE RISK AS TO THE QUALITY AND PERFORMANCE OF THE PROGRAM IS WITH YOU. SHOULD THE PROGRAM PROVE DEFECTIVE, YOU ASSUME THE COST OF ALL NECESSARY SERVICING, REPAIR OR CORRECTION.
+
+12. IN NO EVENT UNLESS REQUIRED BY APPLICABLE LAW OR AGREED TO IN WRITING WILL ANY COPYRIGHT HOLDER, OR ANY OTHER PARTY WHO MAY MODIFY AND/OR REDISTRIBUTE THE PROGRAM AS PERMITTED ABOVE, BE LIABLE TO YOU FOR DAMAGES, INCLUDING ANY GENERAL, SPECIAL, INCIDENTAL OR CONSEQUENTIAL DAMAGES ARISING OUT OF THE USE OR INABILITY TO USE THE PROGRAM (INCLUDING BUT NOT LIMITED TO LOSS OF DATA OR DATA BEING RENDERED INACCURATE OR LOSSES SUSTAINED BY YOU OR THIRD PARTIES OR A FAILURE OF THE PROGRAM TO OPERATE WITH ANY OTHER PROGRAMS), EVEN IF SUCH HOLDER OR OTHER PARTY HAS BEEN ADVISED OF THE POSSIBILITY OF SUCH DAMAGES.
+
+END OF TERMS AND CONDITIONS
+
+How to Apply These Terms to Your New Programs
+
+If you develop a new program, and you want it to be of the greatest possible use to the public, the best way to achieve this is to make it free software which everyone can redistribute and change under these terms.
+
+To do so, attach the following notices to the program. It is safest to attach them to the start of each source file to most effectively convey the exclusion of warranty; and each file should have at least the "copyright" line and a pointer to where the full notice is found.
+
+One line to give the program's name and a brief idea of what it does.
+Copyright (C) <year> <name of author>
+This program is free software; you can redistribute it and/or modify it under the terms of the GNU General Public License as published by the Free Software Foundation; either version 2 of the License, or (at your option) any later version.
+This program is distributed in the hope that it will be useful, but WITHOUT ANY WARRANTY; without even the implied warranty of MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the GNU General Public License for more details.
+You should have received a copy of the GNU General Public License along with this program; if not, write to the Free Software Foundation, Inc., 59 Temple Place, Suite 330, Boston, MA 02111-1307 USA
+Also add information on how to contact you by electronic and paper mail.
+
+If the program is interactive, make it output a short notice like this when it starts in an interactive mode:
+
+Gnomovision version 69, Copyright (C) year name of author Gnomovision comes with ABSOLUTELY NO WARRANTY; for details type `show w'. This is free software, and you are welcome to redistribute it under certain conditions; type `show c' for details.
+The hypothetical commands `show w' and `show c' should show the appropriate parts of the General Public License. Of course, the commands you use may be called something other than `show w' and `show c'; they could even be mouse-clicks or menu items--whatever suits your program.
+
+You should also get your employer (if you work as a programmer) or your school, if any, to sign a "copyright disclaimer" for the program, if necessary. Here is a sample; alter the names:
+
+Yoyodyne, Inc., hereby disclaims all copyright interest in the program `Gnomovision' (which makes passes at compilers) written by James Hacker.
+signature of Ty Coon, 1 April 1989
+Ty Coon, President of Vice
+This General Public License does not permit incorporating your program into proprietary programs. If your program is a subroutine library, you may consider it more useful to permit linking proprietary applications with the library. If this is what you want to do, use the GNU Library General Public License instead of this License.
+
+"CLASSPATH" EXCEPTION TO THE GPL VERSION 2
+
+Certain source files distributed by Oracle are subject to the following clarification and special exception to the GPL Version 2, but only where Oracle has expressly included in the particular source file's header the words "Oracle designates this particular file as subject to the "Classpath" exception as provided by Oracle in the License file that accompanied this code." 
+
+Linking this library statically or dynamically with other modules is making a combined work based on this library.  Thus, the terms and conditions of the GNU General Public License Version 2 cover the whole combination. 
+
+As a special exception, the copyright holders of this library give you permission to link this library with independent modules to produce an executable, regardless of the license terms of these independent modules, and to copy and distribute the resulting executable under terms of your choice, provided that you also meet, for each linked independent module, the terms and conditions of the license of that module.  An independent module is a module which is not derived from or based on this library.  If you modify this library, you may extend this exception to your version of the library, but you are not obligated to do so.  If you do not wish to do so, delete this exception statement from your version.
+
+
+
+
diff --git a/licenses/LICENSE-jline.txt b/licenses/LICENSE-jline.txt
new file mode 100644
index 0000000..9524440
--- /dev/null
+++ b/licenses/LICENSE-jline.txt
@@ -0,0 +1,3 @@
+JLine is distributed under the BSD license, meaning that you are completely free to redistribute, modify, or sell it with almost no restrictins. For more information on the BSD license, see http://www.opensource.org/licenses/bsd-license.php.
+
+
diff --git a/licenses/LICENSE-joni.txt b/licenses/LICENSE-joni.txt
new file mode 100644
index 0000000..bc6ee53
--- /dev/null
+++ b/licenses/LICENSE-joni.txt
@@ -0,0 +1,8 @@
+The MIT License (MIT)
+Copyright (c) <year> <copyright holders>
+
+Permission is hereby granted, free of charge, to any person obtaining a copy of this software and associated documentation files (the "Software"), to deal in the Software without restriction, including without limitation the rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and to permit persons to whom the Software is furnished to do so, subject to the following conditions:
+
+The above copyright notice and this permission notice shall be included in all copies or substantial portions of the Software.
+
+THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE.
diff --git a/licenses/LICENSE-jopt-simple.txt b/licenses/LICENSE-jopt-simple.txt
new file mode 100644
index 0000000..ed192c0
--- /dev/null
+++ b/licenses/LICENSE-jopt-simple.txt
@@ -0,0 +1,22 @@
+The MIT License (MIT)
+
+Permission is hereby granted, free of charge, to any person
+obtaining a copy of this software and associated documentation
+files (the "Software"), to deal in the Software without
+restriction, including without limitation the rights to use,
+copy, modify, merge, publish, distribute, sublicense, and/or sell
+copies of the Software, and to permit persons to whom the
+Software is furnished to do so, subject to the following
+conditions:
+
+The above copyright notice and this permission notice shall be
+included in all copies or substantial portions of the Software.
+
+THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND,
+EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES
+OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND
+NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT
+HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY,
+WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING
+FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR
+OTHER DEALINGS IN THE SOFTWARE.
\ No newline at end of file
diff --git a/licenses/LICENSE-jquery-cookie.txt b/licenses/LICENSE-jquery-cookie.txt
new file mode 100644
index 0000000..7a631e8
--- /dev/null
+++ b/licenses/LICENSE-jquery-cookie.txt
@@ -0,0 +1,20 @@
+Copyright 2014 Klaus Hartl
+
+Permission is hereby granted, free of charge, to any person obtaining
+a copy of this software and associated documentation files (the
+"Software"), to deal in the Software without restriction, including
+without limitation the rights to use, copy, modify, merge, publish,
+distribute, sublicense, and/or sell copies of the Software, and to
+permit persons to whom the Software is furnished to do so, subject to
+the following conditions:
+
+The above copyright notice and this permission notice shall be
+included in all copies or substantial portions of the Software.
+
+THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND,
+EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF
+MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND
+NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE
+LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION
+OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION
+WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE.
diff --git a/licenses/LICENSE-jquery.txt b/licenses/LICENSE-jquery.txt
index fad28aa..5312a4c 100644
--- a/licenses/LICENSE-jquery.txt
+++ b/licenses/LICENSE-jquery.txt
@@ -1,21 +1,36 @@
-# The MIT License (MIT)
+Copyright jQuery Foundation and other contributors, https://jquery.org/
 
-Copyright (c) 2015 jQuery Foundation, Inc.
+This software consists of voluntary contributions made by many
+individuals. For exact contribution history, see the revision history
+available at https://github.com/jquery/jquery
 
-Permission is hereby granted, free of charge, to any person obtaining a copy
-of this software and associated documentation files (the "Software"), to deal
-in the Software without restriction, including without limitation the rights
-to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
-copies of the Software, and to permit persons to whom the Software is
-furnished to do so, subject to the following conditions:
+The following license applies to all parts of this software except as
+documented below:
 
-The above copyright notice and this permission notice shall be included in
-all copies or substantial portions of the Software.
+====
 
-THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
-IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
-FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
-AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
-LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
-OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN
-THE SOFTWARE.
+Permission is hereby granted, free of charge, to any person obtaining
+a copy of this software and associated documentation files (the
+"Software"), to deal in the Software without restriction, including
+without limitation the rights to use, copy, modify, merge, publish,
+distribute, sublicense, and/or sell copies of the Software, and to
+permit persons to whom the Software is furnished to do so, subject to
+the following conditions:
+
+The above copyright notice and this permission notice shall be
+included in all copies or substantial portions of the Software.
+
+THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND,
+EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF
+MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND
+NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE
+LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION
+OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION
+WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE.
+
+====
+
+All files located in the node_modules and external directories are
+externally maintained libraries used by this software which have their
+own licenses; we recommend you read them, as their terms may differ from
+the terms above.
diff --git a/licenses/LICENSE-jsch.txt b/licenses/LICENSE-jsch.txt
new file mode 100644
index 0000000..303096b
--- /dev/null
+++ b/licenses/LICENSE-jsch.txt
@@ -0,0 +1,30 @@
+JSch 0.0.* was released under the GNU LGPL license.  Later, we have switched 
+over to a BSD-style license. 
+
+------------------------------------------------------------------------------
+Copyright (c) 2002-2015 Atsuhiko Yamanaka, JCraft,Inc. 
+All rights reserved.
+
+Redistribution and use in source and binary forms, with or without
+modification, are permitted provided that the following conditions are met:
+
+  1. Redistributions of source code must retain the above copyright notice,
+     this list of conditions and the following disclaimer.
+
+  2. Redistributions in binary form must reproduce the above copyright 
+     notice, this list of conditions and the following disclaimer in 
+     the documentation and/or other materials provided with the distribution.
+
+  3. The names of the authors may not be used to endorse or promote products
+     derived from this software without specific prior written permission.
+
+THIS SOFTWARE IS PROVIDED ``AS IS'' AND ANY EXPRESSED OR IMPLIED WARRANTIES,
+INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND
+FITNESS FOR A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL JCRAFT,
+INC. OR ANY CONTRIBUTORS TO THIS SOFTWARE BE LIABLE FOR ANY DIRECT, INDIRECT,
+INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA,
+OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF
+LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING
+NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE,
+EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
diff --git a/licenses/LICENSE-jsp-api.txt b/licenses/LICENSE-jsp-api.txt
new file mode 100644
index 0000000..bcb65a3
--- /dev/null
+++ b/licenses/LICENSE-jsp-api.txt
@@ -0,0 +1,308 @@
+COMMON DEVELOPMENT AND DISTRIBUTION LICENSE (CDDL) Version 1.0
+1. Definitions.
+
+1.1. Contributor means each individual or entity
+that creates or contributes to the creation of
+Modifications.
+
+1.2. Contributor Version means the combination of the
+Original Software, prior Modifications used by a Contributor (if any), and the
+Modifications made by that particular Contributor.
+
+1.3. Covered
+Software means (a) the Original Software, or (b) Modifications, or (c) the
+combination of files containing Original Software with files containing
+Modifications, in each case including portions
+thereof.
+
+1.4. Executable means the Covered Software in any form other
+than Source Code.
+
+1.5. Initial Developer means the individual or entity
+that first makes Original Software available under this
+License.
+
+1.6. Larger Work means a work which combines Covered Software or
+portions thereof with code not governed by the terms of this
+License.
+
+1.7. License means this document.
+
+1.8. Licensable means
+having the right to grant, to the maximum extent possible, whether at the time
+of the initial grant or subsequently acquired, any and all of the rights
+conveyed herein.
+
+1.9. Modifications means the Source Code and Executable
+form of any of the following:
+A. Any file that results from an addition to,
+deletion from or modification of the contents of a file containing Original
+Software or previous Modifications;
+B. Any new file that contains any part of the Original Software
+or previous Modification; or
+C. Any new file that is contributed or otherwise made available
+under the terms of this License.
+
+1.10. Original Software means the Source Code and Executable form of
+computer software code that is originally released under this License.
+
+1.11. Patent Claims means any patent claim(s), now owned or
+hereafter acquired, including without limitation, method, process, and apparatus
+claims, in any patent Licensable by grantor.
+
+1.12. Source Code means (a) the common form of computer software code in which
+modifications are made and (b) associated documentation included in or
+with such code.
+
+1.13. You (or Your) means an individual or a legal entity exercising rights
+under, and complying with all of the terms of, this License. For legal entities,
+You includes any entity which controls, is controlled by, or is under common control
+with You. For purposes of this definition, control means (a) the power, direct
+or indirect, to cause the direction or management of such entity, whether by
+contract or otherwise, or (b) ownership of more than fifty percent (50%) of the
+outstanding shares or beneficial ownership of such entity.
+
+2. License Grants.
+
+2.1. The Initial Developer Grant. Conditioned upon Your compliance
+with Section 3.1 below and subject to third party intellectual property claims,
+the Initial Developer hereby grants You a world-wide, royalty-free,
+non-exclusive license:
+
+(a) under intellectual property rights (other than
+patent or trademark) Licensable by Initial Developer, to use, reproduce, modify,
+display, perform, sublicense and distribute the Original Software (or portions
+thereof), with or without Modifications, and/or as part of a Larger Work;
+and
+
+(b) under Patent Claims infringed by the making, using or selling of
+Original Software, to make, have made, use, practice, sell, and offer for sale,
+and/or otherwise dispose of the Original Software (or portions
+thereof);
+
+(c) The licenses granted in Sections 2.1(a) and (b) are
+effective on the date Initial Developer first distributes or otherwise makes the
+Original Software available to a third party under the terms of this
+License;
+
+(d) Notwithstanding Section 2.1(b) above, no patent license is
+granted: (1) for code that You delete from the Original Software, or (2) for
+infringements caused by: (i) the modification of the Original Software, or
+(ii) the combination of the Original Software with other software or
+devices.
+
+2.2. Contributor Grant. Conditioned upon Your compliance with
+Section 3.1 below and subject to third party intellectual property claims, each
+Contributor hereby grants You a world-wide, royalty-free, non-exclusive
+license:
+
+(a) under intellectual property rights (other than patent or
+trademark) Licensable by Contributor to use, reproduce, modify, display,
+perform, sublicense and distribute the Modifications created by such Contributor
+(or portions thereof), either on an unmodified basis, with other Modifications,
+as Covered Software and/or as part of a Larger Work; and
+
+(b) under Patent
+Claims infringed by the making, using, or selling of Modifications made by that
+Contributor either alone and/or in combination with its Contributor Version (or
+portions of such combination), to make, use, sell, offer for sale, have made,
+and/or otherwise dispose of: (1) Modifications made by that Contributor (or
+portions thereof); and (2) the combination of Modifications made by that
+Contributor with its Contributor Version (or portions of such
+combination).
+
+(c) The licenses granted in Sections 2.2(a) and 2.2(b) are
+effective on the date Contributor first distributes or otherwise makes the
+Modifications available to a third party.
+
+(d) Notwithstanding Section 2.2(b)
+above, no patent license is granted: (1) for any code that Contributor has
+deleted from the Contributor Version; (2) for infringements caused by:
+(i) third party modifications of Contributor Version, or (ii) the combination
+of Modifications made by that Contributor with other software (except as part of
+the Contributor Version) or other devices; or (3) under Patent Claims infringed
+by Covered Software in the absence of Modifications made by that
+Contributor.
+
+3. Distribution Obligations.
+
+3.1. Availability of Source
+Code. Any Covered Software that You distribute or otherwise make available in
+Executable form must also be made available in Source Code form and that Source
+Code form must be distributed only under the terms of this License. You must
+include a copy of this License with every copy of the Source Code form of the
+Covered Software You distribute or otherwise make available. You must inform
+recipients of any such Covered Software in Executable form as to how they can
+obtain such Covered Software in Source Code form in a reasonable manner on or
+through a medium customarily used for software exchange.
+
+3.2.
+Modifications. The Modifications that You create or to which You contribute are
+governed by the terms of this License. You represent that You believe Your
+Modifications are Your original creation(s) and/or You have sufficient rights to
+grant the rights conveyed by this License.
+
+3.3. Required Notices. You must
+include a notice in each of Your Modifications that identifies You as the
+Contributor of the Modification. You may not remove or alter any copyright,
+patent or trademark notices contained within the Covered Software, or any
+notices of licensing or any descriptive text giving attribution to any
+Contributor or the Initial Developer.
+
+3.4. Application of Additional Terms.
+You may not offer or impose any terms on any Covered Software in Source Code
+form that alters or restricts the applicable version of this License or the
+recipients rights hereunder. You may choose to offer, and to charge a fee for,
+warranty, support, indemnity or liability obligations to one or more recipients
+of Covered Software. However, you may do so only on Your own behalf, and not on
+behalf of the Initial Developer or any Contributor. You must make it absolutely
+clear that any such warranty, support, indemnity or liability obligation is
+offered by You alone, and You hereby agree to indemnify the Initial Developer
+and every Contributor for any liability incurred by the Initial Developer or
+such Contributor as a result of warranty, support, indemnity or liability terms
+You offer.
+
+3.5. Distribution of Executable Versions. You may distribute the
+Executable form of the Covered Software under the terms of this License or under
+the terms of a license of Your choice, which may contain terms different from
+this License, provided that You are in compliance with the terms of this License
+and that the license for the Executable form does not attempt to limit or alter
+the recipients rights in the Source Code form from the rights set forth in this
+License. If You distribute the Covered Software in Executable form under a
+different license, You must make it absolutely clear that any terms which differ
+from this License are offered by You alone, not by the Initial Developer or
+Contributor. You hereby agree to indemnify the Initial Developer and every
+Contributor for any liability incurred by the Initial Developer or such
+Contributor as a result of any such terms You offer.
+
+3.6. Larger Works. You
+may create a Larger Work by combining Covered Software with other code not
+governed by the terms of this License and distribute the Larger Work as a single
+product. In such a case, You must make sure the requirements of this License are
+fulfilled for the Covered Software.
+
+4. Versions of the License.
+
+4.1.
+New Versions. Sun Microsystems, Inc. is the initial license steward and may
+publish revised and/or new versions of this License from time to time. Each
+version will be given a distinguishing version number. Except as provided in
+Section 4.3, no one other than the license steward has the right to modify this
+License.
+
+4.2. Effect of New Versions. You may always continue to use,
+distribute or otherwise make the Covered Software available under the terms of
+the version of the License under which You originally received the Covered
+Software. If the Initial Developer includes a notice in the Original Software
+prohibiting it from being distributed or otherwise made available under any
+subsequent version of the License, You must distribute and make the Covered
+Software available under the terms of the version of the License under which You
+originally received the Covered Software. Otherwise, You may also choose to use,
+distribute or otherwise make the Covered Software available under the terms of
+any subsequent version of the License published by the license
+steward.
+
+4.3. Modified Versions. When You are an Initial Developer and You
+want to create a new license for Your Original Software, You may create and use
+a modified version of this License if You: (a) rename the license and remove
+any references to the name of the license steward (except to note that the
+license differs from this License); and (b) otherwise make it clear that the
+license contains terms which differ from this License.
+
+5. DISCLAIMER OF WARRANTY.
+
+COVERED SOFTWARE IS PROVIDED UNDER THIS LICENSE ON AN AS IS BASIS,
+WITHOUT WARRANTY OF ANY KIND, EITHER EXPRESSED OR IMPLIED, INCLUDING, WITHOUT
+LIMITATION, WARRANTIES THAT THE COVERED SOFTWARE IS FREE OF DEFECTS,
+MERCHANTABLE, FIT FOR A PARTICULAR PURPOSE OR NON-INFRINGING. THE ENTIRE RISK AS
+TO THE QUALITY AND PERFORMANCE OF THE COVERED SOFTWARE IS WITH YOU. SHOULD ANY
+COVERED SOFTWARE PROVE DEFECTIVE IN ANY RESPECT, YOU (NOT THE INITIAL DEVELOPER
+OR ANY OTHER CONTRIBUTOR) ASSUME THE COST OF ANY NECESSARY SERVICING, REPAIR OR
+CORRECTION. THIS DISCLAIMER OF WARRANTY CONSTITUTES AN ESSENTIAL PART OF THIS
+LICENSE. NO USE OF ANY COVERED SOFTWARE IS AUTHORIZED HEREUNDER EXCEPT UNDER
+THIS DISCLAIMER.
+
+6. TERMINATION.
+
+6.1. This License and the rights
+granted hereunder will terminate automatically if You fail to comply with terms
+herein and fail to cure such breach within 30 days of becoming aware of the
+breach. Provisions which, by their nature, must remain in effect beyond the
+termination of this License shall survive.
+
+6.2. If You assert a patent
+infringement claim (excluding declaratory judgment actions) against Initial
+Developer or a Contributor (the Initial Developer or Contributor against whom
+You assert such claim is referred to as Participant) alleging that the
+Participant Software (meaning the Contributor Version where the Participant is a
+Contributor or the Original Software where the Participant is the Initial
+Developer) directly or indirectly infringes any patent, then any and all rights
+granted directly or indirectly to You by such Participant, the Initial Developer
+(if the Initial Developer is not the Participant) and all Contributors under
+Sections 2.1 and/or 2.2 of this License shall, upon 60 days notice from
+Participant terminate prospectively and automatically at the expiration of such
+60 day notice period, unless if within such 60 day period You withdraw Your
+claim with respect to the Participant Software against such Participant either
+unilaterally or pursuant to a written agreement with Participant.
+
+6.3. In
+the event of termination under Sections 6.1 or 6.2 above, all end user licenses
+that have been validly granted by You or any distributor hereunder prior to
+termination (excluding licenses granted to You by any distributor) shall survive
+termination.
+
+7. LIMITATION OF LIABILITY.
+UNDER NO CIRCUMSTANCES AND UNDER
+NO LEGAL THEORY, WHETHER TORT (INCLUDING NEGLIGENCE), CONTRACT, OR OTHERWISE,
+SHALL YOU, THE INITIAL DEVELOPER, ANY OTHER CONTRIBUTOR, OR ANY DISTRIBUTOR OF
+COVERED SOFTWARE, OR ANY SUPPLIER OF ANY OF SUCH PARTIES, BE LIABLE TO ANY
+PERSON FOR ANY INDIRECT, SPECIAL, INCIDENTAL, OR CONSEQUENTIAL DAMAGES OF ANY
+CHARACTER INCLUDING, WITHOUT LIMITATION, DAMAGES FOR LOST PROFITS, LOSS OF
+GOODWILL, WORK STOPPAGE, COMPUTER FAILURE OR MALFUNCTION, OR ANY AND ALL OTHER
+COMMERCIAL DAMAGES OR LOSSES, EVEN IF SUCH PARTY SHALL HAVE BEEN INFORMED OF THE
+POSSIBILITY OF SUCH DAMAGES. THIS LIMITATION OF LIABILITY SHALL NOT APPLY TO
+LIABILITY FOR DEATH OR PERSONAL INJURY RESULTING FROM SUCH PARTYS NEGLIGENCE TO
+THE EXTENT APPLICABLE LAW PROHIBITS SUCH LIMITATION. SOME JURISDICTIONS DO NOT
+ALLOW THE EXCLUSION OR LIMITATION OF INCIDENTAL OR CONSEQUENTIAL DAMAGES, SO
+THIS EXCLUSION AND LIMITATION MAY NOT APPLY TO YOU.
+
+8. U.S. GOVERNMENT END USERS.
+
+The Covered Software is a commercial item, as that term is defined in
+48 C.F.R. 2.101 (Oct. 1995), consisting of commercial computer software (as
+that term is defined at 48 C.F.R.  252.227-7014(a)(1)) and commercial computer
+software documentation as such terms are used in 48 C.F.R. 12.212 (Sept.
+1995). Consistent with 48 C.F.R. 12.212 and 48 C.F.R. 227.7202-1 through
+227.7202-4 (June 1995), all U.S. Government End Users acquire Covered Software
+with only those rights set forth herein. This U.S. Government Rights clause is
+in lieu of, and supersedes, any other FAR, DFAR, or other clause or provision
+that addresses Government rights in computer software under this
+License.
+
+9. MISCELLANEOUS.
+This License represents the complete agreement
+concerning subject matter hereof. If any provision of this License is held to be
+unenforceable, such provision shall be reformed only to the extent necessary to
+make it enforceable. This License shall be governed by the law of the
+jurisdiction specified in a notice contained within the Original Software
+(except to the extent applicable law, if any, provides otherwise), excluding
+such jurisdictions conflict-of-law provisions. Any litigation relating to this
+License shall be subject to the jurisdiction of the courts located in the
+jurisdiction and venue specified in a notice contained within the Original
+Software, with the losing party responsible for costs, including, without
+limitation, court costs and reasonable attorneys fees and expenses. The
+application of the United Nations Convention on Contracts for the International
+Sale of Goods is expressly excluded. Any law or regulation which provides that
+the language of a contract shall be construed against the drafter shall not
+apply to this License. You agree that You alone are responsible for compliance
+with the United States export administration regulations (and the export control
+laws and regulation of any other countries) when You use, distribute or
+otherwise make available any Covered Software.
+
+10. RESPONSIBILITY FOR CLAIMS.
+As between Initial Developer and the Contributors, each party is
+responsible for claims and damages arising, directly or indirectly, out of its
+utilization of rights under this License and You agree to work with Initial
+Developer and Contributors to distribute such responsibility on an equitable
+basis. Nothing herein is intended or shall be deemed to constitute any admission
+of liability.
diff --git a/licenses/LICENSE-junit.txt b/licenses/LICENSE-junit.txt
new file mode 100644
index 0000000..7c3ea8b
--- /dev/null
+++ b/licenses/LICENSE-junit.txt
@@ -0,0 +1,3 @@
+n order to avoid inconsistency the Agreement is copyrighted and may only be modified in the following manner. The Agreement Steward reserves the right to publish new versions (including revisions) of this Agreement from time to time. No one other than the Agreement Steward has the right to modify this Agreement. The Eclipse Foundation is the initial Agreement Steward. The Eclipse Foundation may assign the responsibility to serve as the Agreement Steward to a suitable separate entity. Each new version of the Agreement will be given a distinguishing version number. The Program (including Contributions) may always be distributed subject to the version of the Agreement under which it was received. In addition, after a new version of the Agreement is published, Contributor may elect to distribute the Program (including its Contributions) under the new version. Except as expressly stated in Sections 2(a) and 2(b) above, Recipient receives no rights or licenses to the intellectual property of any Contributor under this Agreement, whether expressly, by implication, estoppel or otherwise. All rights in the Program not expressly granted under this Agreement are reserved.
+
+This Agreement is governed by the laws of the State of New York and the intellectual property laws of the United States of America. No party to this Agreement will bring a legal action under this Agreement more than one year after the cause of action arose. Each party waives its rights to a jury trial in any resulting litigation.
diff --git a/licenses/LICENSE-kryo.txt b/licenses/LICENSE-kryo.txt
new file mode 100644
index 0000000..e1cd884
--- /dev/null
+++ b/licenses/LICENSE-kryo.txt
@@ -0,0 +1,10 @@
+Copyright (c) 2008, Nathan Sweet
+All rights reserved.
+
+Redistribution and use in source and binary forms, with or without modification, are permitted provided that the following conditions are met:
+
+    * Redistributions of source code must retain the above copyright notice, this list of conditions and the following disclaimer.
+    * Redistributions in binary form must reproduce the above copyright notice, this list of conditions and the following disclaimer in the documentation and/or other materials provided with the distribution.
+    * Neither the name of Esoteric Software nor the names of its contributors may be used to endorse or promote products derived from this software without specific prior written permission.
+
+THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
diff --git a/licenses/LICENSE-lodash.txt b/licenses/LICENSE-lodash.txt
new file mode 100644
index 0000000..c6f2f61
--- /dev/null
+++ b/licenses/LICENSE-lodash.txt
@@ -0,0 +1,47 @@
+Copyright JS Foundation and other contributors <https://js.foundation/>
+
+Based on Underscore.js, copyright Jeremy Ashkenas,
+DocumentCloud and Investigative Reporters & Editors <http://underscorejs.org/>
+
+This software consists of voluntary contributions made by many
+individuals. For exact contribution history, see the revision history
+available at https://github.com/lodash/lodash
+
+The following license applies to all parts of this software except as
+documented below:
+
+====
+
+Permission is hereby granted, free of charge, to any person obtaining
+a copy of this software and associated documentation files (the
+"Software"), to deal in the Software without restriction, including
+without limitation the rights to use, copy, modify, merge, publish,
+distribute, sublicense, and/or sell copies of the Software, and to
+permit persons to whom the Software is furnished to do so, subject to
+the following conditions:
+
+The above copyright notice and this permission notice shall be
+included in all copies or substantial portions of the Software.
+
+THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND,
+EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF
+MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND
+NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE
+LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION
+OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION
+WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE.
+
+====
+
+Copyright and related rights for sample code are waived via CC0. Sample
+code is defined as all source code displayed within the prose of the
+documentation.
+
+CC0: http://creativecommons.org/publicdomain/zero/1.0/
+
+====
+
+Files located in the node_modules and vendor directories are externally
+maintained libraries used by this software which have their own
+licenses; we recommend you read them, as their terms may differ from the
+terms above.
diff --git a/licenses/LICENSE-mimepull.txt b/licenses/LICENSE-mimepull.txt
new file mode 100644
index 0000000..1507ec7
--- /dev/null
+++ b/licenses/LICENSE-mimepull.txt
@@ -0,0 +1,210 @@
+COMMON DEVELOPMENT AND DISTRIBUTION LICENSE (CDDL)Version 1.1
+
+1. Definitions.
+
+1.1. “Contributor” means each individual or entity that creates or contributes to the creation of Modifications.
+1.2. “Contributor Version” means the combination of the Original Software, prior Modifications used by a Contributor (if any), and the Modifications made by that particular Contributor.
+1.3. “Covered Software” means (a) the Original Software, or (b) Modifications, or (c) the combination of files containing Original Software with files containing Modifications, in each case including portions thereof.
+1.4. “Executable” means the Covered Software in any form other than Source Code.
+1.5. “Initial Developer” means the individual or entity that first makes Original Software available under this License.
+1.6. “Larger Work” means a work which combines Covered Software or portions thereof with code not governed by the terms of this License.
+1.7. “License” means this document.
+1.8. “Licensable” means having the right to grant, to the maximum extent possible, whether at the time of the initial grant or subsequently acquired, any and all of the rights conveyed herein.
+1.9. “Modifications” means the Source Code and Executable form of any of the following:
+A. Any file that results from an addition to, deletion from or modification of the contents of a file containing Original Software or previous Modifications;
+B. Any new file that contains any part of the Original Software or previous Modification; or
+C. Any new file that is contributed or otherwise made available under the terms of this License.
+1.10. “Original Software” means the Source Code and Executable form of computer software code that is originally released under this License.
+1.11. “Patent Claims” means any patent claim(s), now owned or hereafter acquired, including without limitation, method, process, and apparatus claims, in any patent Licensable by grantor.
+1.12. “Source Code” means (a) the common form of computer software code in which modifications are made and (b) associated documentation included in or with such code.
+1.13. “You” (or “Your”) means an individual or a legal entity exercising rights under, and complying with all of the terms of, this License. For legal entities, “You” includes any entity which controls, is controlled by, or is under common control with You. For purposes of this definition, “control” means (a) the power, direct or indirect, to cause the direction or management of such entity, whether by contract or otherwise, or (b) ownership of more than fifty percent (50%) of the outstanding shares or beneficial ownership of such entity.
+2. License Grants.
+
+2.1. The Initial Developer Grant.
+Conditioned upon Your compliance with Section 3.1 below and subject to third party intellectual property claims, the Initial Developer hereby grants You a world-wide, royalty-free, non-exclusive license:
+(a) under intellectual property rights (other than patent or trademark) Licensable by Initial Developer, to use, reproduce, modify, display, perform, sublicense and distribute the Original Software (or portions thereof), with or without Modifications, and/or as part of a Larger Work; and
+(b) under Patent Claims infringed by the making, using or selling of Original Software, to make, have made, use, practice, sell, and offer for sale, and/or otherwise dispose of the Original Software (or portions thereof).
+(c) The licenses granted in Sections 2.1(a) and (b) are effective on the date Initial Developer first distributes or otherwise makes the Original Software available to a third party under the terms of this License.
+(d) Notwithstanding Section 2.1(b) above, no patent license is granted: (1) for code that You delete from the Original Software, or (2) for infringements caused by: (i) the modification of the Original Software, or (ii) the combination of the Original Software with other software or devices.
+2.2. Contributor Grant.
+Conditioned upon Your compliance with Section 3.1 below and subject to third party intellectual property claims, each Contributor hereby grants You a world-wide, royalty-free, non-exclusive license:
+(a) under intellectual property rights (other than patent or trademark) Licensable by Contributor to use, reproduce, modify, display, perform, sublicense and distribute the Modifications created by such Contributor (or portions thereof), either on an unmodified basis, with other Modifications, as Covered Software and/or as part of a Larger Work; and
+(b) under Patent Claims infringed by the making, using, or selling of Modifications made by that Contributor either alone and/or in combination with its Contributor Version (or portions of such combination), to make, use, sell, offer for sale, have made, and/or otherwise dispose of: (1) Modifications made by that Contributor (or portions thereof); and (2) the combination of Modifications made by that Contributor with its Contributor Version (or portions of such combination).
+(c) The licenses granted in Sections 2.2(a) and 2.2(b) are effective on the date Contributor first distributes or otherwise makes the Modifications available to a third party.
+(d) Notwithstanding Section 2.2(b) above, no patent license is granted: (1) for any code that Contributor has deleted from the Contributor Version; (2) for infringements caused by: (i) third party modifications of Contributor Version, or (ii) the combination of Modifications made by that Contributor with other software (except as part of the Contributor Version) or other devices; or (3) under Patent Claims infringed by Covered Software in the absence of Modifications made by that Contributor.
+3. Distribution Obligations.
+
+3.1. Availability of Source Code.
+Any Covered Software that You distribute or otherwise make available in Executable form must also be made available in Source Code form and that Source Code form must be distributed only under the terms of this License. You must include a copy of this License with every copy of the Source Code form of the Covered Software You distribute or otherwise make available. You must inform recipients of any such Covered Software in Executable form as to how they can obtain such Covered Software in Source Code form in a reasonable manner on or through a medium customarily used for software exchange.
+3.2. Modifications.
+The Modifications that You create or to which You contribute are governed by the terms of this License. You represent that You believe Your Modifications are Your original creation(s) and/or You have sufficient rights to grant the rights conveyed by this License.
+3.3. Required Notices.
+You must include a notice in each of Your Modifications that identifies You as the Contributor of the Modification. You may not remove or alter any copyright, patent or trademark notices contained within the Covered Software, or any notices of licensing or any descriptive text giving attribution to any Contributor or the Initial Developer.
+3.4. Application of Additional Terms.
+You may not offer or impose any terms on any Covered Software in Source Code form that alters or restricts the applicable version of this License or the recipients' rights hereunder. You may choose to offer, and to charge a fee for, warranty, support, indemnity or liability obligations to one or more recipients of Covered Software. However, you may do so only on Your own behalf, and not on behalf of the Initial Developer or any Contributor. You must make it absolutely clear that any such warranty, support, indemnity or liability obligation is offered by You alone, and You hereby agree to indemnify the Initial Developer and every Contributor for any liability incurred by the Initial Developer or such Contributor as a result of warranty, support, indemnity or liability terms You offer.
+3.5. Distribution of Executable Versions.
+You may distribute the Executable form of the Covered Software under the terms of this License or under the terms of a license of Your choice, which may contain terms different from this License, provided that You are in compliance with the terms of this License and that the license for the Executable form does not attempt to limit or alter the recipient's rights in the Source Code form from the rights set forth in this License. If You distribute the Covered Software in Executable form under a different license, You must make it absolutely clear that any terms which differ from this License are offered by You alone, not by the Initial Developer or Contributor. You hereby agree to indemnify the Initial Developer and every Contributor for any liability incurred by the Initial Developer or such Contributor as a result of any such terms You offer.
+3.6. Larger Works.
+You may create a Larger Work by combining Covered Software with other code not governed by the terms of this License and distribute the Larger Work as a single product. In such a case, You must make sure the requirements of this License are fulfilled for the Covered Software.
+4. Versions of the License.
+
+4.1. New Versions.
+Oracle is the initial license steward and may publish revised and/or new versions of this License from time to time. Each version will be given a distinguishing version number. Except as provided in Section 4.3, no one other than the license steward has the right to modify this License.
+4.2. Effect of New Versions.
+You may always continue to use, distribute or otherwise make the Covered Software available under the terms of the version of the License under which You originally received the Covered Software. If the Initial Developer includes a notice in the Original Software prohibiting it from being distributed or otherwise made available under any subsequent version of the License, You must distribute and make the Covered Software available under the terms of the version of the License under which You originally received the Covered Software. Otherwise, You may also choose to use, distribute or otherwise make the Covered Software available under the terms of any subsequent version of the License published by the license steward.
+4.3. Modified Versions.
+When You are an Initial Developer and You want to create a new license for Your Original Software, You may create and use a modified version of this License if You: (a) rename the license and remove any references to the name of the license steward (except to note that the license differs from this License); and (b) otherwise make it clear that the license contains terms which differ from this License.
+5. DISCLAIMER OF WARRANTY.
+
+COVERED SOFTWARE IS PROVIDED UNDER THIS LICENSE ON AN “AS IS” BASIS, WITHOUT WARRANTY OF ANY KIND, EITHER EXPRESSED OR IMPLIED, INCLUDING, WITHOUT LIMITATION, WARRANTIES THAT THE COVERED SOFTWARE IS FREE OF DEFECTS, MERCHANTABLE, FIT FOR A PARTICULAR PURPOSE OR NON-INFRINGING. THE ENTIRE RISK AS TO THE QUALITY AND PERFORMANCE OF THE COVERED SOFTWARE IS WITH YOU. SHOULD ANY COVERED SOFTWARE PROVE DEFECTIVE IN ANY RESPECT, YOU (NOT THE INITIAL DEVELOPER OR ANY OTHER CONTRIBUTOR) ASSUME THE COST OF ANY NECESSARY SERVICING, REPAIR OR CORRECTION. THIS DISCLAIMER OF WARRANTY CONSTITUTES AN ESSENTIAL PART OF THIS LICENSE. NO USE OF ANY COVERED SOFTWARE IS AUTHORIZED HEREUNDER EXCEPT UNDER THIS DISCLAIMER.
+
+6. TERMINATION.
+
+6.1. This License and the rights granted hereunder will terminate automatically if You fail to comply with terms herein and fail to cure such breach within 30 days of becoming aware of the breach. Provisions which, by their nature, must remain in effect beyond the termination of this License shall survive.
+6.2. If You assert a patent infringement claim (excluding declaratory judgment actions) against Initial Developer or a Contributor (the Initial Developer or Contributor against whom You assert such claim is referred to as “Participant”) alleging that the Participant Software (meaning the Contributor Version where the Participant is a Contributor or the Original Software where the Participant is the Initial Developer) directly or indirectly infringes any patent, then any and all rights granted directly or indirectly to You by such Participant, the Initial Developer (if the Initial Developer is not the Participant) and all Contributors under Sections 2.1 and/or 2.2 of this License shall, upon 60 days notice from Participant terminate prospectively and automatically at the expiration of such 60 day notice period, unless if within such 60 day period You withdraw Your claim with respect to the Participant Software against such Participant either unilaterally or pursuant to a written agreement with Participant.
+6.3. If You assert a patent infringement claim against Participant alleging that the Participant Software directly or indirectly infringes any patent where such claim is resolved (such as by license or settlement) prior to the initiation of patent infringement litigation, then the reasonable value of the licenses granted by such Participant under Sections 2.1 or 2.2 shall be taken into account in determining the amount or value of any payment or license.
+6.4. In the event of termination under Sections 6.1 or 6.2 above, all end user licenses that have been validly granted by You or any distributor hereunder prior to termination (excluding licenses granted to You by any distributor) shall survive termination.
+7. LIMITATION OF LIABILITY.
+
+UNDER NO CIRCUMSTANCES AND UNDER NO LEGAL THEORY, WHETHER TORT (INCLUDING NEGLIGENCE), CONTRACT, OR OTHERWISE, SHALL YOU, THE INITIAL DEVELOPER, ANY OTHER CONTRIBUTOR, OR ANY DISTRIBUTOR OF COVERED SOFTWARE, OR ANY SUPPLIER OF ANY OF SUCH PARTIES, BE LIABLE TO ANY PERSON FOR ANY INDIRECT, SPECIAL, INCIDENTAL, OR CONSEQUENTIAL DAMAGES OF ANY CHARACTER INCLUDING, WITHOUT LIMITATION, DAMAGES FOR LOSS OF GOODWILL, WORK STOPPAGE, COMPUTER FAILURE OR MALFUNCTION, OR ANY AND ALL OTHER COMMERCIAL DAMAGES OR LOSSES, EVEN IF SUCH PARTY SHALL HAVE BEEN INFORMED OF THE POSSIBILITY OF SUCH DAMAGES. THIS LIMITATION OF LIABILITY SHALL NOT APPLY TO LIABILITY FOR DEATH OR PERSONAL INJURY RESULTING FROM SUCH PARTY'S NEGLIGENCE TO THE EXTENT APPLICABLE LAW PROHIBITS SUCH LIMITATION. SOME JURISDICTIONS DO NOT ALLOW THE EXCLUSION OR LIMITATION OF INCIDENTAL OR CONSEQUENTIAL DAMAGES, SO THIS EXCLUSION AND LIMITATION MAY NOT APPLY TO YOU.
+
+8. U.S. GOVERNMENT END USERS.
+
+The Covered Software is a “commercial item,” as that term is defined in 48 C.F.R. 2.101 (Oct. 1995), consisting of “commercial computer software” (as that term is defined at 48 C.F.R. § 252.227-7014(a)(1)) and “commercial computer software documentation” as such terms are used in 48 C.F.R. 12.212 (Sept. 1995). Consistent with 48 C.F.R. 12.212 and 48 C.F.R. 227.7202-1 through 227.7202-4 (June 1995), all U.S. Government End Users acquire Covered Software with only those rights set forth herein. This U.S. Government Rights clause is in lieu of, and supersedes, any other FAR, DFAR, or other clause or provision that addresses Government rights in computer software under this License.
+
+9. MISCELLANEOUS.
+
+This License represents the complete agreement concerning subject matter hereof. If any provision of this License is held to be unenforceable, such provision shall be reformed only to the extent necessary to make it enforceable. This License shall be governed by the law of the jurisdiction specified in a notice contained within the Original Software (except to the extent applicable law, if any, provides otherwise), excluding such jurisdiction's conflict-of-law provisions. Any litigation relating to this License shall be subject to the jurisdiction of the courts located in the jurisdiction and venue specified in a notice contained within the Original Software, with the losing party responsible for costs, including, without limitation, court costs and reasonable attorneys' fees and expenses. The application of the United Nations Convention on Contracts for the International Sale of Goods is expressly excluded. Any law or regulation which provides that the language of a contract shall be construed against the drafter shall not apply to this License. You agree that You alone are responsible for compliance with the United States export administration regulations (and the export control laws and regulation of any other countries) when You use, distribute or otherwise make available any Covered Software.
+
+10. RESPONSIBILITY FOR CLAIMS.
+
+As between Initial Developer and the Contributors, each party is responsible for claims and damages arising, directly or indirectly, out of its utilization of rights under this License and You agree to work with Initial Developer and Contributors to distribute such responsibility on an equitable basis. Nothing herein is intended or shall be deemed to constitute any admission of liability.
+
+NOTICE PURSUANT TO SECTION 9 OF THE COMMON DEVELOPMENT AND DISTRIBUTION LICENSE (CDDL)
+
+The code released under the CDDL shall be governed by the laws of the State of California (excluding conflict-of-law provisions). Any litigation relating to this License shall be subject to the jurisdiction of the Federal Courts of the Northern District of California and the state courts of the State of California, with venue lying in Santa Clara County, California.
+
+
+
+The GNU General Public License (GPL) Version 2, June 1991
+
+Copyright (C) 1989, 1991 Free Software Foundation, Inc.
+59 Temple Place, Suite 330, Boston, MA 02111-1307 USA
+
+Everyone is permitted to copy and distribute verbatim copies
+of this license document, but changing it is not allowed.
+
+Preamble
+
+The licenses for most software are designed to take away your freedom to share and change it. By contrast, the GNU General Public License is intended to guarantee your freedom to share and change free software--to make sure the software is free for all its users. This General Public License applies to most of the Free Software Foundation's software and to any other program whose authors commit to using it. (Some other Free Software Foundation software is covered by the GNU Library General Public License instead.) You can apply it to your programs, too.
+
+When we speak of free software, we are referring to freedom, not price. Our General Public Licenses are designed to make sure that you have the freedom to distribute copies of free software (and charge for this service if you wish), that you receive source code or can get it if you want it, that you can change the software or use pieces of it in new free programs; and that you know you can do these things.
+
+To protect your rights, we need to make restrictions that forbid anyone to deny you these rights or to ask you to surrender the rights. These restrictions translate to certain responsibilities for you if you distribute copies of the software, or if you modify it.
+
+For example, if you distribute copies of such a program, whether gratis or for a fee, you must give the recipients all the rights that you have. You must make sure that they, too, receive or can get the source code. And you must show them these terms so they know their rights.
+
+We protect your rights with two steps: (1) copyright the software, and (2) offer you this license which gives you legal permission to copy, distribute and/or modify the software.
+
+Also, for each author's protection and ours, we want to make certain that everyone understands that there is no warranty for this free software. If the software is modified by someone else and passed on, we want its recipients to know that what they have is not the original, so that any problems introduced by others will not reflect on the original authors' reputations.
+
+Finally, any free program is threatened constantly by software patents. We wish to avoid the danger that redistributors of a free program will individually obtain patent licenses, in effect making the program proprietary. To prevent this, we have made it clear that any patent must be licensed for everyone's free use or not licensed at all.
+
+The precise terms and conditions for copying, distribution and modification follow.
+
+TERMS AND CONDITIONS FOR COPYING, DISTRIBUTION AND MODIFICATION
+
+0. This License applies to any program or other work which contains a notice placed by the copyright holder saying it may be distributed under the terms of this General Public License. The "Program", below, refers to any such program or work, and a "work based on the Program" means either the Program or any derivative work under copyright law: that is to say, a work containing the Program or a portion of it, either verbatim or with modifications and/or translated into another language. (Hereinafter, translation is included without limitation in the term "modification".) Each licensee is addressed as "you".
+
+Activities other than copying, distribution and modification are not covered by this License; they are outside its scope. The act of running the Program is not restricted, and the output from the Program is covered only if its contents constitute a work based on the Program (independent of having been made by running the Program). Whether that is true depends on what the Program does.
+
+1. You may copy and distribute verbatim copies of the Program's source code as you receive it, in any medium, provided that you conspicuously and appropriately publish on each copy an appropriate copyright notice and disclaimer of warranty; keep intact all the notices that refer to this License and to the absence of any warranty; and give any other recipients of the Program a copy of this License along with the Program.
+
+You may charge a fee for the physical act of transferring a copy, and you may at your option offer warranty protection in exchange for a fee.
+
+2. You may modify your copy or copies of the Program or any portion of it, thus forming a work based on the Program, and copy and distribute such modifications or work under the terms of Section 1 above, provided that you also meet all of these conditions:
+
+a) You must cause the modified files to carry prominent notices stating that you changed the files and the date of any change.
+b) You must cause any work that you distribute or publish, that in whole or in part contains or is derived from the Program or any part thereof, to be licensed as a whole at no charge to all third parties under the terms of this License.
+c) If the modified program normally reads commands interactively when run, you must cause it, when started running for such interactive use in the most ordinary way, to print or display an announcement including an appropriate copyright notice and a notice that there is no warranty (or else, saying that you provide a warranty) and that users may redistribute the program under these conditions, and telling the user how to view a copy of this License. (Exception: if the Program itself is interactive but does not normally print such an announcement, your work based on the Program is not required to print an announcement.)
+These requirements apply to the modified work as a whole. If identifiable sections of that work are not derived from the Program, and can be reasonably considered independent and separate works in themselves, then this License, and its terms, do not apply to those sections when you distribute them as separate works. But when you distribute the same sections as part of a whole which is a work based on the Program, the distribution of the whole must be on the terms of this License, whose permissions for other licensees extend to the entire whole, and thus to each and every part regardless of who wrote it.
+
+Thus, it is not the intent of this section to claim rights or contest your rights to work written entirely by you; rather, the intent is to exercise the right to control the distribution of derivative or collective works based on the Program.
+
+In addition, mere aggregation of another work not based on the Program with the Program (or with a work based on the Program) on a volume of a storage or distribution medium does not bring the other work under the scope of this License.
+
+3. You may copy and distribute the Program (or a work based on it, under Section 2) in object code or executable form under the terms of Sections 1 and 2 above provided that you also do one of the following:
+
+a) Accompany it with the complete corresponding machine-readable source code, which must be distributed under the terms of Sections 1 and 2 above on a medium customarily used for software interchange; or,
+b) Accompany it with a written offer, valid for at least three years, to give any third party, for a charge no more than your cost of physically performing source distribution, a complete machine-readable copy of the corresponding source code, to be distributed under the terms of Sections 1 and 2 above on a medium customarily used for software interchange; or,
+c) Accompany it with the information you received as to the offer to distribute corresponding source code. (This alternative is allowed only for noncommercial distribution and only if you received the program in object code or executable form with such an offer, in accord with Subsection b above.)
+The source code for a work means the preferred form of the work for making modifications to it. For an executable work, complete source code means all the source code for all modules it contains, plus any associated interface definition files, plus the scripts used to control compilation and installation of the executable. However, as a special exception, the source code distributed need not include anything that is normally distributed (in either source or binary form) with the major components (compiler, kernel, and so on) of the operating system on which the executable runs, unless that component itself accompanies the executable.
+
+If distribution of executable or object code is made by offering access to copy from a designated place, then offering equivalent access to copy the source code from the same place counts as distribution of the source code, even though third parties are not compelled to copy the source along with the object code.
+
+4. You may not copy, modify, sublicense, or distribute the Program except as expressly provided under this License. Any attempt otherwise to copy, modify, sublicense or distribute the Program is void, and will automatically terminate your rights under this License. However, parties who have received copies, or rights, from you under this License will not have their licenses terminated so long as such parties remain in full compliance.
+
+5. You are not required to accept this License, since you have not signed it. However, nothing else grants you permission to modify or distribute the Program or its derivative works. These actions are prohibited by law if you do not accept this License. Therefore, by modifying or distributing the Program (or any work based on the Program), you indicate your acceptance of this License to do so, and all its terms and conditions for copying, distributing or modifying the Program or works based on it.
+
+6. Each time you redistribute the Program (or any work based on the Program), the recipient automatically receives a license from the original licensor to copy, distribute or modify the Program subject to these terms and conditions. You may not impose any further restrictions on the recipients' exercise of the rights granted herein. You are not responsible for enforcing compliance by third parties to this License.
+
+7. If, as a consequence of a court judgment or allegation of patent infringement or for any other reason (not limited to patent issues), conditions are imposed on you (whether by court order, agreement or otherwise) that contradict the conditions of this License, they do not excuse you from the conditions of this License. If you cannot distribute so as to satisfy simultaneously your obligations under this License and any other pertinent obligations, then as a consequence you may not distribute the Program at all. For example, if a patent license would not permit royalty-free redistribution of the Program by all those who receive copies directly or indirectly through you, then the only way you could satisfy both it and this License would be to refrain entirely from distribution of the Program.
+
+If any portion of this section is held invalid or unenforceable under any particular circumstance, the balance of the section is intended to apply and the section as a whole is intended to apply in other circumstances.
+
+It is not the purpose of this section to induce you to infringe any patents or other property right claims or to contest validity of any such claims; this section has the sole purpose of protecting the integrity of the free software distribution system, which is implemented by public license practices. Many people have made generous contributions to the wide range of software distributed through that system in reliance on consistent application of that system; it is up to the author/donor to decide if he or she is willing to distribute software through any other system and a licensee cannot impose that choice.
+
+This section is intended to make thoroughly clear what is believed to be a consequence of the rest of this License.
+
+8. If the distribution and/or use of the Program is restricted in certain countries either by patents or by copyrighted interfaces, the original copyright holder who places the Program under this License may add an explicit geographical distribution limitation excluding those countries, so that distribution is permitted only in or among countries not thus excluded. In such case, this License incorporates the limitation as if written in the body of this License.
+
+9. The Free Software Foundation may publish revised and/or new versions of the General Public License from time to time. Such new versions will be similar in spirit to the present version, but may differ in detail to address new problems or concerns.
+
+Each version is given a distinguishing version number. If the Program specifies a version number of this License which applies to it and "any later version", you have the option of following the terms and conditions either of that version or of any later version published by the Free Software Foundation. If the Program does not specify a version number of this License, you may choose any version ever published by the Free Software Foundation.
+
+10. If you wish to incorporate parts of the Program into other free programs whose distribution conditions are different, write to the author to ask for permission. For software which is copyrighted by the Free Software Foundation, write to the Free Software Foundation; we sometimes make exceptions for this. Our decision will be guided by the two goals of preserving the free status of all derivatives of our free software and of promoting the sharing and reuse of software generally.
+
+NO WARRANTY
+
+11. BECAUSE THE PROGRAM IS LICENSED FREE OF CHARGE, THERE IS NO WARRANTY FOR THE PROGRAM, TO THE EXTENT PERMITTED BY APPLICABLE LAW. EXCEPT WHEN OTHERWISE STATED IN WRITING THE COPYRIGHT HOLDERS AND/OR OTHER PARTIES PROVIDE THE PROGRAM "AS IS" WITHOUT WARRANTY OF ANY KIND, EITHER EXPRESSED OR IMPLIED, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE. THE ENTIRE RISK AS TO THE QUALITY AND PERFORMANCE OF THE PROGRAM IS WITH YOU. SHOULD THE PROGRAM PROVE DEFECTIVE, YOU ASSUME THE COST OF ALL NECESSARY SERVICING, REPAIR OR CORRECTION.
+
+12. IN NO EVENT UNLESS REQUIRED BY APPLICABLE LAW OR AGREED TO IN WRITING WILL ANY COPYRIGHT HOLDER, OR ANY OTHER PARTY WHO MAY MODIFY AND/OR REDISTRIBUTE THE PROGRAM AS PERMITTED ABOVE, BE LIABLE TO YOU FOR DAMAGES, INCLUDING ANY GENERAL, SPECIAL, INCIDENTAL OR CONSEQUENTIAL DAMAGES ARISING OUT OF THE USE OR INABILITY TO USE THE PROGRAM (INCLUDING BUT NOT LIMITED TO LOSS OF DATA OR DATA BEING RENDERED INACCURATE OR LOSSES SUSTAINED BY YOU OR THIRD PARTIES OR A FAILURE OF THE PROGRAM TO OPERATE WITH ANY OTHER PROGRAMS), EVEN IF SUCH HOLDER OR OTHER PARTY HAS BEEN ADVISED OF THE POSSIBILITY OF SUCH DAMAGES.
+
+END OF TERMS AND CONDITIONS
+
+How to Apply These Terms to Your New Programs
+
+If you develop a new program, and you want it to be of the greatest possible use to the public, the best way to achieve this is to make it free software which everyone can redistribute and change under these terms.
+
+To do so, attach the following notices to the program. It is safest to attach them to the start of each source file to most effectively convey the exclusion of warranty; and each file should have at least the "copyright" line and a pointer to where the full notice is found.
+
+One line to give the program's name and a brief idea of what it does.
+Copyright (C) <year> <name of author>
+This program is free software; you can redistribute it and/or modify it under the terms of the GNU General Public License as published by the Free Software Foundation; either version 2 of the License, or (at your option) any later version.
+This program is distributed in the hope that it will be useful, but WITHOUT ANY WARRANTY; without even the implied warranty of MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the GNU General Public License for more details.
+You should have received a copy of the GNU General Public License along with this program; if not, write to the Free Software Foundation, Inc., 59 Temple Place, Suite 330, Boston, MA 02111-1307 USA
+Also add information on how to contact you by electronic and paper mail.
+
+If the program is interactive, make it output a short notice like this when it starts in an interactive mode:
+
+Gnomovision version 69, Copyright (C) year name of author Gnomovision comes with ABSOLUTELY NO WARRANTY; for details type `show w'. This is free software, and you are welcome to redistribute it under certain conditions; type `show c' for details.
+The hypothetical commands `show w' and `show c' should show the appropriate parts of the General Public License. Of course, the commands you use may be called something other than `show w' and `show c'; they could even be mouse-clicks or menu items--whatever suits your program.
+
+You should also get your employer (if you work as a programmer) or your school, if any, to sign a "copyright disclaimer" for the program, if necessary. Here is a sample; alter the names:
+
+Yoyodyne, Inc., hereby disclaims all copyright interest in the program `Gnomovision' (which makes passes at compilers) written by James Hacker.
+signature of Ty Coon, 1 April 1989
+Ty Coon, President of Vice
+This General Public License does not permit incorporating your program into proprietary programs. If your program is a subroutine library, you may consider it more useful to permit linking proprietary applications with the library. If this is what you want to do, use the GNU Library General Public License instead of this License.
+
+"CLASSPATH" EXCEPTION TO THE GPL VERSION 2
+
+Certain source files distributed by Oracle are subject to the following clarification and special exception to the GPL Version 2, but only where Oracle has expressly included in the particular source file's header the words "Oracle designates this particular file as subject to the "Classpath" exception as provided by Oracle in the License file that accompanied this code." 
+
+Linking this library statically or dynamically with other modules is making a combined work based on this library.  Thus, the terms and conditions of the GNU General Public License Version 2 cover the whole combination. 
+
+As a special exception, the copyright holders of this library give you permission to link this library with independent modules to produce an executable, regardless of the license terms of these independent modules, and to copy and distribute the resulting executable under terms of your choice, provided that you also meet, for each linked independent module, the terms and conditions of the license of that module.  An independent module is a module which is not derived from or based on this library.  If you modify this library, you may extend this exception to your version of the library, but you are not obligated to do so.  If you do not wish to do so, delete this exception statement from your version.
+
+
+
+
diff --git a/licenses/LICENSE-minlog.txt b/licenses/LICENSE-minlog.txt
new file mode 100644
index 0000000..e1cd884
--- /dev/null
+++ b/licenses/LICENSE-minlog.txt
@@ -0,0 +1,10 @@
+Copyright (c) 2008, Nathan Sweet
+All rights reserved.
+
+Redistribution and use in source and binary forms, with or without modification, are permitted provided that the following conditions are met:
+
+    * Redistributions of source code must retain the above copyright notice, this list of conditions and the following disclaimer.
+    * Redistributions in binary form must reproduce the above copyright notice, this list of conditions and the following disclaimer in the documentation and/or other materials provided with the distribution.
+    * Neither the name of Esoteric Software nor the names of its contributors may be used to endorse or promote products derived from this software without specific prior written permission.
+
+THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
diff --git a/licenses/LICENSE-modernizr.txt b/licenses/LICENSE-modernizr.txt
deleted file mode 100644
index 8a955dd..0000000
--- a/licenses/LICENSE-modernizr.txt
+++ /dev/null
@@ -1,21 +0,0 @@
-# The MIT License (MIT)
-
-Copyright (c) 2009-2016 Faruk Ates, Paul Irish, Alex Sexton, Ryan Seddon, Patrick Kettner, Stu Cox, Richard Herrera
-
-Permission is hereby granted, free of charge, to any person obtaining a copy
-of this software and associated documentation files (the "Software"), to deal
-in the Software without restriction, including without limitation the rights
-to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
-copies of the Software, and to permit persons to whom the Software is
-furnished to do so, subject to the following conditions:
-
-The above copyright notice and this permission notice shall be included in
-all copies or substantial portions of the Software.
-
-THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
-IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
-FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
-AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
-LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
-OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN
-THE SOFTWARE.
diff --git a/licenses/LICENSE-momentjs.txt b/licenses/LICENSE-momentjs.txt
new file mode 100644
index 0000000..8618b73
--- /dev/null
+++ b/licenses/LICENSE-momentjs.txt
@@ -0,0 +1,22 @@
+Copyright (c) JS Foundation and other contributors
+
+Permission is hereby granted, free of charge, to any person
+obtaining a copy of this software and associated documentation
+files (the "Software"), to deal in the Software without
+restriction, including without limitation the rights to use,
+copy, modify, merge, publish, distribute, sublicense, and/or sell
+copies of the Software, and to permit persons to whom the
+Software is furnished to do so, subject to the following
+conditions:
+
+The above copyright notice and this permission notice shall be
+included in all copies or substantial portions of the Software.
+
+THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND,
+EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES
+OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND
+NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT
+HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY,
+WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING
+FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR
+OTHER DEALINGS IN THE SOFTWARE.
diff --git a/licenses/LICENSE-ng-file-upload.txt b/licenses/LICENSE-ng-file-upload.txt
new file mode 100644
index 0000000..7ebd53c
--- /dev/null
+++ b/licenses/LICENSE-ng-file-upload.txt
@@ -0,0 +1,20 @@
+The MIT License (MIT)
+
+Copyright (c) 2013 danialfarid
+
+Permission is hereby granted, free of charge, to any person obtaining a copy of
+this software and associated documentation files (the "Software"), to deal in
+the Software without restriction, including without limitation the rights to
+use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of
+the Software, and to permit persons to whom the Software is furnished to do so,
+subject to the following conditions:
+
+The above copyright notice and this permission notice shall be included in all
+copies or substantial portions of the Software.
+
+THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, FITNESS
+FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR
+COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER
+IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN
+CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE.
diff --git a/licenses/LICENSE-paranamer.txt b/licenses/LICENSE-paranamer.txt
new file mode 100644
index 0000000..522176f
--- /dev/null
+++ b/licenses/LICENSE-paranamer.txt
@@ -0,0 +1,28 @@
+[ ParaNamer used to be 'Pubic Domain', but since it includes a small piece of ASM it is now the same license as that: BSD ]
+
+ Copyright (c) 2006 Paul Hammant & ThoughtWorks Inc
+ All rights reserved.
+
+ Redistribution and use in source and binary forms, with or without
+ modification, are permitted provided that the following conditions
+ are met:
+ 1. Redistributions of source code must retain the above copyright
+    notice, this list of conditions and the following disclaimer.
+ 2. Redistributions in binary form must reproduce the above copyright
+    notice, this list of conditions and the following disclaimer in the
+    documentation and/or other materials provided with the distribution.
+ 3. Neither the name of the copyright holders nor the names of its
+    contributors may be used to endorse or promote products derived from
+    this software without specific prior written permission.
+
+ THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS"
+ AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE
+ IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE
+ ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE
+ LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR
+ CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF
+ SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS
+ INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN
+ CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE)
+ ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF
+ THE POSSIBILITY OF SUCH DAMAGE.
diff --git a/licenses/LICENSE-polyfill.txt b/licenses/LICENSE-polyfill.txt
deleted file mode 100644
index 6b73375..0000000
--- a/licenses/LICENSE-polyfill.txt
+++ /dev/null
@@ -1,21 +0,0 @@
-# The MIT License (MIT)
-
-Copyright (c) 2012 Scott Jehl, Paul Irish, Nicholas Zakas
-
-Permission is hereby granted, free of charge, to any person obtaining a copy
-of this software and associated documentation files (the "Software"), to deal
-in the Software without restriction, including without limitation the rights
-to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
-copies of the Software, and to permit persons to whom the Software is
-furnished to do so, subject to the following conditions:
-
-The above copyright notice and this permission notice shall be included in
-all copies or substantial portions of the Software.
-
-THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
-IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
-FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
-AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
-LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
-OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN
-THE SOFTWARE.
\ No newline at end of file
diff --git a/licenses/LICENSE-protobuf-java.txt b/licenses/LICENSE-protobuf-java.txt
new file mode 100644
index 0000000..f028c82
--- /dev/null
+++ b/licenses/LICENSE-protobuf-java.txt
@@ -0,0 +1,42 @@
+This license applies to all parts of Protocol Buffers except the following:
+
+  - Atomicops support for generic gcc, located in
+    src/google/protobuf/stubs/atomicops_internals_generic_gcc.h.
+    This file is copyrighted by Red Hat Inc.
+
+  - Atomicops support for AIX/POWER, located in
+    src/google/protobuf/stubs/atomicops_internals_power.h.
+    This file is copyrighted by Bloomberg Finance LP.
+
+Copyright 2014, Google Inc.  All rights reserved.
+
+Redistribution and use in source and binary forms, with or without
+modification, are permitted provided that the following conditions are
+met:
+
+    * Redistributions of source code must retain the above copyright
+notice, this list of conditions and the following disclaimer.
+    * Redistributions in binary form must reproduce the above
+copyright notice, this list of conditions and the following disclaimer
+in the documentation and/or other materials provided with the
+distribution.
+    * Neither the name of Google Inc. nor the names of its
+contributors may be used to endorse or promote products derived from
+this software without specific prior written permission.
+
+THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+
+Code generated by the Protocol Buffer compiler is owned by the owner
+of the input file used when generating it.  This code is not
+standalone and requires a support library to be linked with it.  This
+support library is itself covered by the above license.
diff --git a/licenses/LICENSE-reactive-streams.txt b/licenses/LICENSE-reactive-streams.txt
new file mode 100644
index 0000000..bb5bc37
--- /dev/null
+++ b/licenses/LICENSE-reactive-streams.txt
@@ -0,0 +1,9 @@
+Licensed under Public Domain (CC0)
+
+To the extent possible under law, the person who associated CC0 with
+this code has waived all copyright and related or neighboring
+rights to this code.
+
+You should have received a copy of the CC0 legalcode along with this
+work. If not, see <http://creativecommons.org/publicdomain/zero/1.0/>.
+
diff --git a/licenses/LICENSE-reflectasm.txt b/licenses/LICENSE-reflectasm.txt
new file mode 100644
index 0000000..e1cd884
--- /dev/null
+++ b/licenses/LICENSE-reflectasm.txt
@@ -0,0 +1,10 @@
+Copyright (c) 2008, Nathan Sweet
+All rights reserved.
+
+Redistribution and use in source and binary forms, with or without modification, are permitted provided that the following conditions are met:
+
+    * Redistributions of source code must retain the above copyright notice, this list of conditions and the following disclaimer.
+    * Redistributions in binary form must reproduce the above copyright notice, this list of conditions and the following disclaimer in the documentation and/or other materials provided with the distribution.
+    * Neither the name of Esoteric Software nor the names of its contributors may be used to endorse or promote products derived from this software without specific prior written permission.
+
+THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
diff --git a/licenses/LICENSE-scala-java8-compat.txt b/licenses/LICENSE-scala-java8-compat.txt
new file mode 100644
index 0000000..fb7f26f
--- /dev/null
+++ b/licenses/LICENSE-scala-java8-compat.txt
@@ -0,0 +1,14 @@
+Scala is licensed under the BSD 3-Clause License.
+
+Scala License
+Copyright (c) 2002-2016 EPFL
+Copyright (c) 2011-2016 Lightbend, Inc. (formerly Typesafe, Inc.)
+
+All rights reserved.
+
+Redistribution and use in source and binary forms, with or without modification, are permitted provided that the following conditions are met:
+
+Redistributions of source code must retain the above copyright notice, this list of conditions and the following disclaimer.
+Redistributions in binary form must reproduce the above copyright notice, this list of conditions and the following disclaimer in the documentation and/or other materials provided with the distribution.
+Neither the name of the EPFL nor the names of its contributors may be used to endorse or promote products derived from this software without specific prior written permission.
+THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS “AS IS” AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
diff --git a/licenses/LICENSE-scala-parser-combinators.txt b/licenses/LICENSE-scala-parser-combinators.txt
new file mode 100644
index 0000000..fb7f26f
--- /dev/null
+++ b/licenses/LICENSE-scala-parser-combinators.txt
@@ -0,0 +1,14 @@
+Scala is licensed under the BSD 3-Clause License.
+
+Scala License
+Copyright (c) 2002-2016 EPFL
+Copyright (c) 2011-2016 Lightbend, Inc. (formerly Typesafe, Inc.)
+
+All rights reserved.
+
+Redistribution and use in source and binary forms, with or without modification, are permitted provided that the following conditions are met:
+
+Redistributions of source code must retain the above copyright notice, this list of conditions and the following disclaimer.
+Redistributions in binary form must reproduce the above copyright notice, this list of conditions and the following disclaimer in the documentation and/or other materials provided with the distribution.
+Neither the name of the EPFL nor the names of its contributors may be used to endorse or promote products derived from this software without specific prior written permission.
+THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS “AS IS” AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
diff --git a/licenses/LICENSE-scala-reflect.txt b/licenses/LICENSE-scala-reflect.txt
new file mode 100644
index 0000000..fb7f26f
--- /dev/null
+++ b/licenses/LICENSE-scala-reflect.txt
@@ -0,0 +1,14 @@
+Scala is licensed under the BSD 3-Clause License.
+
+Scala License
+Copyright (c) 2002-2016 EPFL
+Copyright (c) 2011-2016 Lightbend, Inc. (formerly Typesafe, Inc.)
+
+All rights reserved.
+
+Redistribution and use in source and binary forms, with or without modification, are permitted provided that the following conditions are met:
+
+Redistributions of source code must retain the above copyright notice, this list of conditions and the following disclaimer.
+Redistributions in binary form must reproduce the above copyright notice, this list of conditions and the following disclaimer in the documentation and/or other materials provided with the distribution.
+Neither the name of the EPFL nor the names of its contributors may be used to endorse or promote products derived from this software without specific prior written permission.
+THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS “AS IS” AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
diff --git a/licenses/LICENSE-scala-stm.txt b/licenses/LICENSE-scala-stm.txt
new file mode 100644
index 0000000..c7bb7b7
--- /dev/null
+++ b/licenses/LICENSE-scala-stm.txt
@@ -0,0 +1,23 @@
+All of the source code and documentation that has been developed by the Scala STM Expert Group is available under a modified-BSD license.
+Copyright (c) 2009-2012 Stanford University, unless otherwise specified.
+All rights reserved.
+Redistribution and use in source and binary forms, with or without
+modification, are permitted provided that the following conditions are met:
+    * Redistributions of source code must retain the above copyright
+      notice, this list of conditions and the following disclaimer.
+    * Redistributions in binary form must reproduce the above copyright
+      notice, this list of conditions and the following disclaimer in the
+      documentation and/or other materials provided with the distribution.
+    * Neither the name of the Stanford University nor the
+      names of its contributors may be used to endorse or promote products
+      derived from this software without specific prior written permission.
+THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND
+ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED
+WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE
+DISCLAIMED. IN NO EVENT SHALL STANFORD UNIVERSITY BE LIABLE FOR ANY
+DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES
+(INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES;
+LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND
+ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS
+SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
diff --git a/licenses/LICENSE-normalize.txt b/licenses/LICENSE-scribejava-apis.txt
similarity index 92%
copy from licenses/LICENSE-normalize.txt
copy to licenses/LICENSE-scribejava-apis.txt
index ce68d94..a50928b 100644
--- a/licenses/LICENSE-normalize.txt
+++ b/licenses/LICENSE-scribejava-apis.txt
@@ -1,6 +1,7 @@
-# The MIT License (MIT)
+The MIT License
 
-Copyright © Nicolas Gallagher and Jonathan Neal
+Copyright (c) 2013 hh.ru
+Copyright (c) 2010 Pablo Fernandez
 
 Permission is hereby granted, free of charge, to any person obtaining a copy
 of this software and associated documentation files (the "Software"), to deal
@@ -18,4 +19,4 @@
 AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
 LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
 OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN
-THE SOFTWARE.
\ No newline at end of file
+THE SOFTWARE.
diff --git a/licenses/LICENSE-normalize.txt b/licenses/LICENSE-scribejava-core.txt
similarity index 92%
copy from licenses/LICENSE-normalize.txt
copy to licenses/LICENSE-scribejava-core.txt
index ce68d94..a50928b 100644
--- a/licenses/LICENSE-normalize.txt
+++ b/licenses/LICENSE-scribejava-core.txt
@@ -1,6 +1,7 @@
-# The MIT License (MIT)
+The MIT License
 
-Copyright © Nicolas Gallagher and Jonathan Neal
+Copyright (c) 2013 hh.ru
+Copyright (c) 2010 Pablo Fernandez
 
 Permission is hereby granted, free of charge, to any person obtaining a copy
 of this software and associated documentation files (the "Software"), to deal
@@ -18,4 +19,4 @@
 AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
 LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
 OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN
-THE SOFTWARE.
\ No newline at end of file
+THE SOFTWARE.
diff --git a/licenses/LICENSE-servlet-api.txt b/licenses/LICENSE-servlet-api.txt
new file mode 100644
index 0000000..bcb65a3
--- /dev/null
+++ b/licenses/LICENSE-servlet-api.txt
@@ -0,0 +1,308 @@
+COMMON DEVELOPMENT AND DISTRIBUTION LICENSE (CDDL) Version 1.0
+1. Definitions.
+
+1.1. Contributor means each individual or entity
+that creates or contributes to the creation of
+Modifications.
+
+1.2. Contributor Version means the combination of the
+Original Software, prior Modifications used by a Contributor (if any), and the
+Modifications made by that particular Contributor.
+
+1.3. Covered
+Software means (a) the Original Software, or (b) Modifications, or (c) the
+combination of files containing Original Software with files containing
+Modifications, in each case including portions
+thereof.
+
+1.4. Executable means the Covered Software in any form other
+than Source Code.
+
+1.5. Initial Developer means the individual or entity
+that first makes Original Software available under this
+License.
+
+1.6. Larger Work means a work which combines Covered Software or
+portions thereof with code not governed by the terms of this
+License.
+
+1.7. License means this document.
+
+1.8. Licensable means
+having the right to grant, to the maximum extent possible, whether at the time
+of the initial grant or subsequently acquired, any and all of the rights
+conveyed herein.
+
+1.9. Modifications means the Source Code and Executable
+form of any of the following:
+A. Any file that results from an addition to,
+deletion from or modification of the contents of a file containing Original
+Software or previous Modifications;
+B. Any new file that contains any part of the Original Software
+or previous Modification; or
+C. Any new file that is contributed or otherwise made available
+under the terms of this License.
+
+1.10. Original Software means the Source Code and Executable form of
+computer software code that is originally released under this License.
+
+1.11. Patent Claims means any patent claim(s), now owned or
+hereafter acquired, including without limitation, method, process, and apparatus
+claims, in any patent Licensable by grantor.
+
+1.12. Source Code means (a) the common form of computer software code in which
+modifications are made and (b) associated documentation included in or
+with such code.
+
+1.13. You (or Your) means an individual or a legal entity exercising rights
+under, and complying with all of the terms of, this License. For legal entities,
+You includes any entity which controls, is controlled by, or is under common control
+with You. For purposes of this definition, control means (a) the power, direct
+or indirect, to cause the direction or management of such entity, whether by
+contract or otherwise, or (b) ownership of more than fifty percent (50%) of the
+outstanding shares or beneficial ownership of such entity.
+
+2. License Grants.
+
+2.1. The Initial Developer Grant. Conditioned upon Your compliance
+with Section 3.1 below and subject to third party intellectual property claims,
+the Initial Developer hereby grants You a world-wide, royalty-free,
+non-exclusive license:
+
+(a) under intellectual property rights (other than
+patent or trademark) Licensable by Initial Developer, to use, reproduce, modify,
+display, perform, sublicense and distribute the Original Software (or portions
+thereof), with or without Modifications, and/or as part of a Larger Work;
+and
+
+(b) under Patent Claims infringed by the making, using or selling of
+Original Software, to make, have made, use, practice, sell, and offer for sale,
+and/or otherwise dispose of the Original Software (or portions
+thereof);
+
+(c) The licenses granted in Sections 2.1(a) and (b) are
+effective on the date Initial Developer first distributes or otherwise makes the
+Original Software available to a third party under the terms of this
+License;
+
+(d) Notwithstanding Section 2.1(b) above, no patent license is
+granted: (1) for code that You delete from the Original Software, or (2) for
+infringements caused by: (i) the modification of the Original Software, or
+(ii) the combination of the Original Software with other software or
+devices.
+
+2.2. Contributor Grant. Conditioned upon Your compliance with
+Section 3.1 below and subject to third party intellectual property claims, each
+Contributor hereby grants You a world-wide, royalty-free, non-exclusive
+license:
+
+(a) under intellectual property rights (other than patent or
+trademark) Licensable by Contributor to use, reproduce, modify, display,
+perform, sublicense and distribute the Modifications created by such Contributor
+(or portions thereof), either on an unmodified basis, with other Modifications,
+as Covered Software and/or as part of a Larger Work; and
+
+(b) under Patent
+Claims infringed by the making, using, or selling of Modifications made by that
+Contributor either alone and/or in combination with its Contributor Version (or
+portions of such combination), to make, use, sell, offer for sale, have made,
+and/or otherwise dispose of: (1) Modifications made by that Contributor (or
+portions thereof); and (2) the combination of Modifications made by that
+Contributor with its Contributor Version (or portions of such
+combination).
+
+(c) The licenses granted in Sections 2.2(a) and 2.2(b) are
+effective on the date Contributor first distributes or otherwise makes the
+Modifications available to a third party.
+
+(d) Notwithstanding Section 2.2(b)
+above, no patent license is granted: (1) for any code that Contributor has
+deleted from the Contributor Version; (2) for infringements caused by:
+(i) third party modifications of Contributor Version, or (ii) the combination
+of Modifications made by that Contributor with other software (except as part of
+the Contributor Version) or other devices; or (3) under Patent Claims infringed
+by Covered Software in the absence of Modifications made by that
+Contributor.
+
+3. Distribution Obligations.
+
+3.1. Availability of Source
+Code. Any Covered Software that You distribute or otherwise make available in
+Executable form must also be made available in Source Code form and that Source
+Code form must be distributed only under the terms of this License. You must
+include a copy of this License with every copy of the Source Code form of the
+Covered Software You distribute or otherwise make available. You must inform
+recipients of any such Covered Software in Executable form as to how they can
+obtain such Covered Software in Source Code form in a reasonable manner on or
+through a medium customarily used for software exchange.
+
+3.2.
+Modifications. The Modifications that You create or to which You contribute are
+governed by the terms of this License. You represent that You believe Your
+Modifications are Your original creation(s) and/or You have sufficient rights to
+grant the rights conveyed by this License.
+
+3.3. Required Notices. You must
+include a notice in each of Your Modifications that identifies You as the
+Contributor of the Modification. You may not remove or alter any copyright,
+patent or trademark notices contained within the Covered Software, or any
+notices of licensing or any descriptive text giving attribution to any
+Contributor or the Initial Developer.
+
+3.4. Application of Additional Terms.
+You may not offer or impose any terms on any Covered Software in Source Code
+form that alters or restricts the applicable version of this License or the
+recipients rights hereunder. You may choose to offer, and to charge a fee for,
+warranty, support, indemnity or liability obligations to one or more recipients
+of Covered Software. However, you may do so only on Your own behalf, and not on
+behalf of the Initial Developer or any Contributor. You must make it absolutely
+clear that any such warranty, support, indemnity or liability obligation is
+offered by You alone, and You hereby agree to indemnify the Initial Developer
+and every Contributor for any liability incurred by the Initial Developer or
+such Contributor as a result of warranty, support, indemnity or liability terms
+You offer.
+
+3.5. Distribution of Executable Versions. You may distribute the
+Executable form of the Covered Software under the terms of this License or under
+the terms of a license of Your choice, which may contain terms different from
+this License, provided that You are in compliance with the terms of this License
+and that the license for the Executable form does not attempt to limit or alter
+the recipients rights in the Source Code form from the rights set forth in this
+License. If You distribute the Covered Software in Executable form under a
+different license, You must make it absolutely clear that any terms which differ
+from this License are offered by You alone, not by the Initial Developer or
+Contributor. You hereby agree to indemnify the Initial Developer and every
+Contributor for any liability incurred by the Initial Developer or such
+Contributor as a result of any such terms You offer.
+
+3.6. Larger Works. You
+may create a Larger Work by combining Covered Software with other code not
+governed by the terms of this License and distribute the Larger Work as a single
+product. In such a case, You must make sure the requirements of this License are
+fulfilled for the Covered Software.
+
+4. Versions of the License.
+
+4.1.
+New Versions. Sun Microsystems, Inc. is the initial license steward and may
+publish revised and/or new versions of this License from time to time. Each
+version will be given a distinguishing version number. Except as provided in
+Section 4.3, no one other than the license steward has the right to modify this
+License.
+
+4.2. Effect of New Versions. You may always continue to use,
+distribute or otherwise make the Covered Software available under the terms of
+the version of the License under which You originally received the Covered
+Software. If the Initial Developer includes a notice in the Original Software
+prohibiting it from being distributed or otherwise made available under any
+subsequent version of the License, You must distribute and make the Covered
+Software available under the terms of the version of the License under which You
+originally received the Covered Software. Otherwise, You may also choose to use,
+distribute or otherwise make the Covered Software available under the terms of
+any subsequent version of the License published by the license
+steward.
+
+4.3. Modified Versions. When You are an Initial Developer and You
+want to create a new license for Your Original Software, You may create and use
+a modified version of this License if You: (a) rename the license and remove
+any references to the name of the license steward (except to note that the
+license differs from this License); and (b) otherwise make it clear that the
+license contains terms which differ from this License.
+
+5. DISCLAIMER OF WARRANTY.
+
+COVERED SOFTWARE IS PROVIDED UNDER THIS LICENSE ON AN AS IS BASIS,
+WITHOUT WARRANTY OF ANY KIND, EITHER EXPRESSED OR IMPLIED, INCLUDING, WITHOUT
+LIMITATION, WARRANTIES THAT THE COVERED SOFTWARE IS FREE OF DEFECTS,
+MERCHANTABLE, FIT FOR A PARTICULAR PURPOSE OR NON-INFRINGING. THE ENTIRE RISK AS
+TO THE QUALITY AND PERFORMANCE OF THE COVERED SOFTWARE IS WITH YOU. SHOULD ANY
+COVERED SOFTWARE PROVE DEFECTIVE IN ANY RESPECT, YOU (NOT THE INITIAL DEVELOPER
+OR ANY OTHER CONTRIBUTOR) ASSUME THE COST OF ANY NECESSARY SERVICING, REPAIR OR
+CORRECTION. THIS DISCLAIMER OF WARRANTY CONSTITUTES AN ESSENTIAL PART OF THIS
+LICENSE. NO USE OF ANY COVERED SOFTWARE IS AUTHORIZED HEREUNDER EXCEPT UNDER
+THIS DISCLAIMER.
+
+6. TERMINATION.
+
+6.1. This License and the rights
+granted hereunder will terminate automatically if You fail to comply with terms
+herein and fail to cure such breach within 30 days of becoming aware of the
+breach. Provisions which, by their nature, must remain in effect beyond the
+termination of this License shall survive.
+
+6.2. If You assert a patent
+infringement claim (excluding declaratory judgment actions) against Initial
+Developer or a Contributor (the Initial Developer or Contributor against whom
+You assert such claim is referred to as Participant) alleging that the
+Participant Software (meaning the Contributor Version where the Participant is a
+Contributor or the Original Software where the Participant is the Initial
+Developer) directly or indirectly infringes any patent, then any and all rights
+granted directly or indirectly to You by such Participant, the Initial Developer
+(if the Initial Developer is not the Participant) and all Contributors under
+Sections 2.1 and/or 2.2 of this License shall, upon 60 days notice from
+Participant terminate prospectively and automatically at the expiration of such
+60 day notice period, unless if within such 60 day period You withdraw Your
+claim with respect to the Participant Software against such Participant either
+unilaterally or pursuant to a written agreement with Participant.
+
+6.3. In
+the event of termination under Sections 6.1 or 6.2 above, all end user licenses
+that have been validly granted by You or any distributor hereunder prior to
+termination (excluding licenses granted to You by any distributor) shall survive
+termination.
+
+7. LIMITATION OF LIABILITY.
+UNDER NO CIRCUMSTANCES AND UNDER
+NO LEGAL THEORY, WHETHER TORT (INCLUDING NEGLIGENCE), CONTRACT, OR OTHERWISE,
+SHALL YOU, THE INITIAL DEVELOPER, ANY OTHER CONTRIBUTOR, OR ANY DISTRIBUTOR OF
+COVERED SOFTWARE, OR ANY SUPPLIER OF ANY OF SUCH PARTIES, BE LIABLE TO ANY
+PERSON FOR ANY INDIRECT, SPECIAL, INCIDENTAL, OR CONSEQUENTIAL DAMAGES OF ANY
+CHARACTER INCLUDING, WITHOUT LIMITATION, DAMAGES FOR LOST PROFITS, LOSS OF
+GOODWILL, WORK STOPPAGE, COMPUTER FAILURE OR MALFUNCTION, OR ANY AND ALL OTHER
+COMMERCIAL DAMAGES OR LOSSES, EVEN IF SUCH PARTY SHALL HAVE BEEN INFORMED OF THE
+POSSIBILITY OF SUCH DAMAGES. THIS LIMITATION OF LIABILITY SHALL NOT APPLY TO
+LIABILITY FOR DEATH OR PERSONAL INJURY RESULTING FROM SUCH PARTYS NEGLIGENCE TO
+THE EXTENT APPLICABLE LAW PROHIBITS SUCH LIMITATION. SOME JURISDICTIONS DO NOT
+ALLOW THE EXCLUSION OR LIMITATION OF INCIDENTAL OR CONSEQUENTIAL DAMAGES, SO
+THIS EXCLUSION AND LIMITATION MAY NOT APPLY TO YOU.
+
+8. U.S. GOVERNMENT END USERS.
+
+The Covered Software is a commercial item, as that term is defined in
+48 C.F.R. 2.101 (Oct. 1995), consisting of commercial computer software (as
+that term is defined at 48 C.F.R.  252.227-7014(a)(1)) and commercial computer
+software documentation as such terms are used in 48 C.F.R. 12.212 (Sept.
+1995). Consistent with 48 C.F.R. 12.212 and 48 C.F.R. 227.7202-1 through
+227.7202-4 (June 1995), all U.S. Government End Users acquire Covered Software
+with only those rights set forth herein. This U.S. Government Rights clause is
+in lieu of, and supersedes, any other FAR, DFAR, or other clause or provision
+that addresses Government rights in computer software under this
+License.
+
+9. MISCELLANEOUS.
+This License represents the complete agreement
+concerning subject matter hereof. If any provision of this License is held to be
+unenforceable, such provision shall be reformed only to the extent necessary to
+make it enforceable. This License shall be governed by the law of the
+jurisdiction specified in a notice contained within the Original Software
+(except to the extent applicable law, if any, provides otherwise), excluding
+such jurisdictions conflict-of-law provisions. Any litigation relating to this
+License shall be subject to the jurisdiction of the courts located in the
+jurisdiction and venue specified in a notice contained within the Original
+Software, with the losing party responsible for costs, including, without
+limitation, court costs and reasonable attorneys fees and expenses. The
+application of the United Nations Convention on Contracts for the International
+Sale of Goods is expressly excluded. Any law or regulation which provides that
+the language of a contract shall be construed against the drafter shall not
+apply to this License. You agree that You alone are responsible for compliance
+with the United States export administration regulations (and the export control
+laws and regulation of any other countries) when You use, distribute or
+otherwise make available any Covered Software.
+
+10. RESPONSIBILITY FOR CLAIMS.
+As between Initial Developer and the Contributors, each party is
+responsible for claims and damages arising, directly or indirectly, out of its
+utilization of rights under this License and You agree to work with Initial
+Developer and Contributors to distribute such responsibility on an equitable
+basis. Nothing herein is intended or shall be deemed to constitute any admission
+of liability.
diff --git a/licenses/LICENSE-slf4j.txt b/licenses/LICENSE-slf4j.txt
new file mode 100644
index 0000000..f9f183a
--- /dev/null
+++ b/licenses/LICENSE-slf4j.txt
@@ -0,0 +1,21 @@
+Copyright (c) 2004-2013 QOS.ch
+ All rights reserved.
+
+ Permission is hereby granted, free  of charge, to any person obtaining
+ a  copy  of this  software  and  associated  documentation files  (the
+ "Software"), to  deal in  the Software without  restriction, including
+ without limitation  the rights to  use, copy, modify,  merge, publish,
+ distribute,  sublicense, and/or sell  copies of  the Software,  and to
+ permit persons to whom the Software  is furnished to do so, subject to
+ the following conditions:
+ 
+ The  above  copyright  notice  and  this permission  notice  shall  be
+ included in all copies or substantial portions of the Software.
+ 
+ THE  SOFTWARE IS  PROVIDED  "AS  IS", WITHOUT  WARRANTY  OF ANY  KIND,
+ EXPRESS OR  IMPLIED, INCLUDING  BUT NOT LIMITED  TO THE  WARRANTIES OF
+ MERCHANTABILITY,    FITNESS    FOR    A   PARTICULAR    PURPOSE    AND
+ NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE
+ LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION
+ OF CONTRACT, TORT OR OTHERWISE,  ARISING FROM, OUT OF OR IN CONNECTION
+ WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE.
diff --git a/licenses/LICENSE-stax-api.txt b/licenses/LICENSE-stax-api.txt
new file mode 100644
index 0000000..bcb65a3
--- /dev/null
+++ b/licenses/LICENSE-stax-api.txt
@@ -0,0 +1,308 @@
+COMMON DEVELOPMENT AND DISTRIBUTION LICENSE (CDDL) Version 1.0
+1. Definitions.
+
+1.1. Contributor means each individual or entity
+that creates or contributes to the creation of
+Modifications.
+
+1.2. Contributor Version means the combination of the
+Original Software, prior Modifications used by a Contributor (if any), and the
+Modifications made by that particular Contributor.
+
+1.3. Covered
+Software means (a) the Original Software, or (b) Modifications, or (c) the
+combination of files containing Original Software with files containing
+Modifications, in each case including portions
+thereof.
+
+1.4. Executable means the Covered Software in any form other
+than Source Code.
+
+1.5. Initial Developer means the individual or entity
+that first makes Original Software available under this
+License.
+
+1.6. Larger Work means a work which combines Covered Software or
+portions thereof with code not governed by the terms of this
+License.
+
+1.7. License means this document.
+
+1.8. Licensable means
+having the right to grant, to the maximum extent possible, whether at the time
+of the initial grant or subsequently acquired, any and all of the rights
+conveyed herein.
+
+1.9. Modifications means the Source Code and Executable
+form of any of the following:
+A. Any file that results from an addition to,
+deletion from or modification of the contents of a file containing Original
+Software or previous Modifications;
+B. Any new file that contains any part of the Original Software
+or previous Modification; or
+C. Any new file that is contributed or otherwise made available
+under the terms of this License.
+
+1.10. Original Software means the Source Code and Executable form of
+computer software code that is originally released under this License.
+
+1.11. Patent Claims means any patent claim(s), now owned or
+hereafter acquired, including without limitation, method, process, and apparatus
+claims, in any patent Licensable by grantor.
+
+1.12. Source Code means (a) the common form of computer software code in which
+modifications are made and (b) associated documentation included in or
+with such code.
+
+1.13. You (or Your) means an individual or a legal entity exercising rights
+under, and complying with all of the terms of, this License. For legal entities,
+You includes any entity which controls, is controlled by, or is under common control
+with You. For purposes of this definition, control means (a) the power, direct
+or indirect, to cause the direction or management of such entity, whether by
+contract or otherwise, or (b) ownership of more than fifty percent (50%) of the
+outstanding shares or beneficial ownership of such entity.
+
+2. License Grants.
+
+2.1. The Initial Developer Grant. Conditioned upon Your compliance
+with Section 3.1 below and subject to third party intellectual property claims,
+the Initial Developer hereby grants You a world-wide, royalty-free,
+non-exclusive license:
+
+(a) under intellectual property rights (other than
+patent or trademark) Licensable by Initial Developer, to use, reproduce, modify,
+display, perform, sublicense and distribute the Original Software (or portions
+thereof), with or without Modifications, and/or as part of a Larger Work;
+and
+
+(b) under Patent Claims infringed by the making, using or selling of
+Original Software, to make, have made, use, practice, sell, and offer for sale,
+and/or otherwise dispose of the Original Software (or portions
+thereof);
+
+(c) The licenses granted in Sections 2.1(a) and (b) are
+effective on the date Initial Developer first distributes or otherwise makes the
+Original Software available to a third party under the terms of this
+License;
+
+(d) Notwithstanding Section 2.1(b) above, no patent license is
+granted: (1) for code that You delete from the Original Software, or (2) for
+infringements caused by: (i) the modification of the Original Software, or
+(ii) the combination of the Original Software with other software or
+devices.
+
+2.2. Contributor Grant. Conditioned upon Your compliance with
+Section 3.1 below and subject to third party intellectual property claims, each
+Contributor hereby grants You a world-wide, royalty-free, non-exclusive
+license:
+
+(a) under intellectual property rights (other than patent or
+trademark) Licensable by Contributor to use, reproduce, modify, display,
+perform, sublicense and distribute the Modifications created by such Contributor
+(or portions thereof), either on an unmodified basis, with other Modifications,
+as Covered Software and/or as part of a Larger Work; and
+
+(b) under Patent
+Claims infringed by the making, using, or selling of Modifications made by that
+Contributor either alone and/or in combination with its Contributor Version (or
+portions of such combination), to make, use, sell, offer for sale, have made,
+and/or otherwise dispose of: (1) Modifications made by that Contributor (or
+portions thereof); and (2) the combination of Modifications made by that
+Contributor with its Contributor Version (or portions of such
+combination).
+
+(c) The licenses granted in Sections 2.2(a) and 2.2(b) are
+effective on the date Contributor first distributes or otherwise makes the
+Modifications available to a third party.
+
+(d) Notwithstanding Section 2.2(b)
+above, no patent license is granted: (1) for any code that Contributor has
+deleted from the Contributor Version; (2) for infringements caused by:
+(i) third party modifications of Contributor Version, or (ii) the combination
+of Modifications made by that Contributor with other software (except as part of
+the Contributor Version) or other devices; or (3) under Patent Claims infringed
+by Covered Software in the absence of Modifications made by that
+Contributor.
+
+3. Distribution Obligations.
+
+3.1. Availability of Source
+Code. Any Covered Software that You distribute or otherwise make available in
+Executable form must also be made available in Source Code form and that Source
+Code form must be distributed only under the terms of this License. You must
+include a copy of this License with every copy of the Source Code form of the
+Covered Software You distribute or otherwise make available. You must inform
+recipients of any such Covered Software in Executable form as to how they can
+obtain such Covered Software in Source Code form in a reasonable manner on or
+through a medium customarily used for software exchange.
+
+3.2.
+Modifications. The Modifications that You create or to which You contribute are
+governed by the terms of this License. You represent that You believe Your
+Modifications are Your original creation(s) and/or You have sufficient rights to
+grant the rights conveyed by this License.
+
+3.3. Required Notices. You must
+include a notice in each of Your Modifications that identifies You as the
+Contributor of the Modification. You may not remove or alter any copyright,
+patent or trademark notices contained within the Covered Software, or any
+notices of licensing or any descriptive text giving attribution to any
+Contributor or the Initial Developer.
+
+3.4. Application of Additional Terms.
+You may not offer or impose any terms on any Covered Software in Source Code
+form that alters or restricts the applicable version of this License or the
+recipients rights hereunder. You may choose to offer, and to charge a fee for,
+warranty, support, indemnity or liability obligations to one or more recipients
+of Covered Software. However, you may do so only on Your own behalf, and not on
+behalf of the Initial Developer or any Contributor. You must make it absolutely
+clear that any such warranty, support, indemnity or liability obligation is
+offered by You alone, and You hereby agree to indemnify the Initial Developer
+and every Contributor for any liability incurred by the Initial Developer or
+such Contributor as a result of warranty, support, indemnity or liability terms
+You offer.
+
+3.5. Distribution of Executable Versions. You may distribute the
+Executable form of the Covered Software under the terms of this License or under
+the terms of a license of Your choice, which may contain terms different from
+this License, provided that You are in compliance with the terms of this License
+and that the license for the Executable form does not attempt to limit or alter
+the recipients rights in the Source Code form from the rights set forth in this
+License. If You distribute the Covered Software in Executable form under a
+different license, You must make it absolutely clear that any terms which differ
+from this License are offered by You alone, not by the Initial Developer or
+Contributor. You hereby agree to indemnify the Initial Developer and every
+Contributor for any liability incurred by the Initial Developer or such
+Contributor as a result of any such terms You offer.
+
+3.6. Larger Works. You
+may create a Larger Work by combining Covered Software with other code not
+governed by the terms of this License and distribute the Larger Work as a single
+product. In such a case, You must make sure the requirements of this License are
+fulfilled for the Covered Software.
+
+4. Versions of the License.
+
+4.1.
+New Versions. Sun Microsystems, Inc. is the initial license steward and may
+publish revised and/or new versions of this License from time to time. Each
+version will be given a distinguishing version number. Except as provided in
+Section 4.3, no one other than the license steward has the right to modify this
+License.
+
+4.2. Effect of New Versions. You may always continue to use,
+distribute or otherwise make the Covered Software available under the terms of
+the version of the License under which You originally received the Covered
+Software. If the Initial Developer includes a notice in the Original Software
+prohibiting it from being distributed or otherwise made available under any
+subsequent version of the License, You must distribute and make the Covered
+Software available under the terms of the version of the License under which You
+originally received the Covered Software. Otherwise, You may also choose to use,
+distribute or otherwise make the Covered Software available under the terms of
+any subsequent version of the License published by the license
+steward.
+
+4.3. Modified Versions. When You are an Initial Developer and You
+want to create a new license for Your Original Software, You may create and use
+a modified version of this License if You: (a) rename the license and remove
+any references to the name of the license steward (except to note that the
+license differs from this License); and (b) otherwise make it clear that the
+license contains terms which differ from this License.
+
+5. DISCLAIMER OF WARRANTY.
+
+COVERED SOFTWARE IS PROVIDED UNDER THIS LICENSE ON AN AS IS BASIS,
+WITHOUT WARRANTY OF ANY KIND, EITHER EXPRESSED OR IMPLIED, INCLUDING, WITHOUT
+LIMITATION, WARRANTIES THAT THE COVERED SOFTWARE IS FREE OF DEFECTS,
+MERCHANTABLE, FIT FOR A PARTICULAR PURPOSE OR NON-INFRINGING. THE ENTIRE RISK AS
+TO THE QUALITY AND PERFORMANCE OF THE COVERED SOFTWARE IS WITH YOU. SHOULD ANY
+COVERED SOFTWARE PROVE DEFECTIVE IN ANY RESPECT, YOU (NOT THE INITIAL DEVELOPER
+OR ANY OTHER CONTRIBUTOR) ASSUME THE COST OF ANY NECESSARY SERVICING, REPAIR OR
+CORRECTION. THIS DISCLAIMER OF WARRANTY CONSTITUTES AN ESSENTIAL PART OF THIS
+LICENSE. NO USE OF ANY COVERED SOFTWARE IS AUTHORIZED HEREUNDER EXCEPT UNDER
+THIS DISCLAIMER.
+
+6. TERMINATION.
+
+6.1. This License and the rights
+granted hereunder will terminate automatically if You fail to comply with terms
+herein and fail to cure such breach within 30 days of becoming aware of the
+breach. Provisions which, by their nature, must remain in effect beyond the
+termination of this License shall survive.
+
+6.2. If You assert a patent
+infringement claim (excluding declaratory judgment actions) against Initial
+Developer or a Contributor (the Initial Developer or Contributor against whom
+You assert such claim is referred to as Participant) alleging that the
+Participant Software (meaning the Contributor Version where the Participant is a
+Contributor or the Original Software where the Participant is the Initial
+Developer) directly or indirectly infringes any patent, then any and all rights
+granted directly or indirectly to You by such Participant, the Initial Developer
+(if the Initial Developer is not the Participant) and all Contributors under
+Sections 2.1 and/or 2.2 of this License shall, upon 60 days notice from
+Participant terminate prospectively and automatically at the expiration of such
+60 day notice period, unless if within such 60 day period You withdraw Your
+claim with respect to the Participant Software against such Participant either
+unilaterally or pursuant to a written agreement with Participant.
+
+6.3. In
+the event of termination under Sections 6.1 or 6.2 above, all end user licenses
+that have been validly granted by You or any distributor hereunder prior to
+termination (excluding licenses granted to You by any distributor) shall survive
+termination.
+
+7. LIMITATION OF LIABILITY.
+UNDER NO CIRCUMSTANCES AND UNDER
+NO LEGAL THEORY, WHETHER TORT (INCLUDING NEGLIGENCE), CONTRACT, OR OTHERWISE,
+SHALL YOU, THE INITIAL DEVELOPER, ANY OTHER CONTRIBUTOR, OR ANY DISTRIBUTOR OF
+COVERED SOFTWARE, OR ANY SUPPLIER OF ANY OF SUCH PARTIES, BE LIABLE TO ANY
+PERSON FOR ANY INDIRECT, SPECIAL, INCIDENTAL, OR CONSEQUENTIAL DAMAGES OF ANY
+CHARACTER INCLUDING, WITHOUT LIMITATION, DAMAGES FOR LOST PROFITS, LOSS OF
+GOODWILL, WORK STOPPAGE, COMPUTER FAILURE OR MALFUNCTION, OR ANY AND ALL OTHER
+COMMERCIAL DAMAGES OR LOSSES, EVEN IF SUCH PARTY SHALL HAVE BEEN INFORMED OF THE
+POSSIBILITY OF SUCH DAMAGES. THIS LIMITATION OF LIABILITY SHALL NOT APPLY TO
+LIABILITY FOR DEATH OR PERSONAL INJURY RESULTING FROM SUCH PARTYS NEGLIGENCE TO
+THE EXTENT APPLICABLE LAW PROHIBITS SUCH LIMITATION. SOME JURISDICTIONS DO NOT
+ALLOW THE EXCLUSION OR LIMITATION OF INCIDENTAL OR CONSEQUENTIAL DAMAGES, SO
+THIS EXCLUSION AND LIMITATION MAY NOT APPLY TO YOU.
+
+8. U.S. GOVERNMENT END USERS.
+
+The Covered Software is a commercial item, as that term is defined in
+48 C.F.R. 2.101 (Oct. 1995), consisting of commercial computer software (as
+that term is defined at 48 C.F.R.  252.227-7014(a)(1)) and commercial computer
+software documentation as such terms are used in 48 C.F.R. 12.212 (Sept.
+1995). Consistent with 48 C.F.R. 12.212 and 48 C.F.R. 227.7202-1 through
+227.7202-4 (June 1995), all U.S. Government End Users acquire Covered Software
+with only those rights set forth herein. This U.S. Government Rights clause is
+in lieu of, and supersedes, any other FAR, DFAR, or other clause or provision
+that addresses Government rights in computer software under this
+License.
+
+9. MISCELLANEOUS.
+This License represents the complete agreement
+concerning subject matter hereof. If any provision of this License is held to be
+unenforceable, such provision shall be reformed only to the extent necessary to
+make it enforceable. This License shall be governed by the law of the
+jurisdiction specified in a notice contained within the Original Software
+(except to the extent applicable law, if any, provides otherwise), excluding
+such jurisdictions conflict-of-law provisions. Any litigation relating to this
+License shall be subject to the jurisdiction of the courts located in the
+jurisdiction and venue specified in a notice contained within the Original
+Software, with the losing party responsible for costs, including, without
+limitation, court costs and reasonable attorneys fees and expenses. The
+application of the United Nations Convention on Contracts for the International
+Sale of Goods is expressly excluded. Any law or regulation which provides that
+the language of a contract shall be construed against the drafter shall not
+apply to this License. You agree that You alone are responsible for compliance
+with the United States export administration regulations (and the export control
+laws and regulation of any other countries) when You use, distribute or
+otherwise make available any Covered Software.
+
+10. RESPONSIBILITY FOR CLAIMS.
+As between Initial Developer and the Contributors, each party is
+responsible for claims and damages arising, directly or indirectly, out of its
+utilization of rights under this License and You agree to work with Initial
+Developer and Contributors to distribute such responsibility on an equitable
+basis. Nothing herein is intended or shall be deemed to constitute any admission
+of liability.
diff --git a/licenses/LICENSE-ui-select.txt b/licenses/LICENSE-ui-select.txt
new file mode 100644
index 0000000..b62d482
--- /dev/null
+++ b/licenses/LICENSE-ui-select.txt
@@ -0,0 +1,20 @@
+The MIT License (MIT)
+
+Copyright (c) 2013-2014 AngularUI
+
+Permission is hereby granted, free of charge, to any person obtaining a copy of
+this software and associated documentation files (the "Software"), to deal in
+the Software without restriction, including without limitation the rights to
+use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of
+the Software, and to permit persons to whom the Software is furnished to do so,
+subject to the following conditions:
+
+The above copyright notice and this permission notice shall be included in all
+copies or substantial portions of the Software.
+
+THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, FITNESS
+FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR
+COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER
+IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN
+CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE.
diff --git a/licenses/LICENSE-upickle.txt b/licenses/LICENSE-upickle.txt
new file mode 100644
index 0000000..5f0ccbe
--- /dev/null
+++ b/licenses/LICENSE-upickle.txt
@@ -0,0 +1,7 @@
+Copyright (c) 2016 haoyi.sg@gmail.com
+
+Permission is hereby granted, free of charge, to any person obtaining a copy of this software and associated documentation files (the "Software"), to deal in the Software without restriction, including without limitation the rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and to permit persons to whom the Software is furnished to do so, subject to the following conditions:
+
+The above copyright notice and this permission notice shall be included in all copies or substantial portions of the Software.
+
+THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE.
diff --git a/licenses/LICENSE-xmlenc.txt b/licenses/LICENSE-xmlenc.txt
new file mode 100644
index 0000000..3a70c9b
--- /dev/null
+++ b/licenses/LICENSE-xmlenc.txt
@@ -0,0 +1,27 @@
+Copyright 2003-2005, Ernst de Haan <wfe.dehaan@gmail.com>
+All rights reserved.
+
+Redistribution and use in source and binary forms, with or without
+modification, are permitted provided that the following conditions are met:
+
+1. Redistributions of source code must retain the above copyright notice, this
+   list of conditions and the following disclaimer.
+
+2. Redistributions in binary form must reproduce the above copyright notice,
+   this list of conditions and the following disclaimer in the documentation
+   and/or other materials provided with the distribution.
+
+3. Neither the name of the copyright holder nor the names of its contributors
+   may be used to endorse or promote products derived from this software
+   without specific prior written permission.
+
+THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDER AND CONTRIBUTORS "AS IS"
+AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE
+IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE
+DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE LIABLE
+FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL
+DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR
+SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER
+CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY,
+OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
diff --git a/licenses/LICENSE-xz.txt b/licenses/LICENSE-xz.txt
new file mode 100644
index 0000000..ee50e15
--- /dev/null
+++ b/licenses/LICENSE-xz.txt
@@ -0,0 +1 @@
+This Java implementation of XZ has been put into the public domain, thus you can do whatever you want with it. All the files in the package have been written by Lasse Collin, but some files are heavily based on public domain code written by Igor Pavlov.
diff --git a/notes/0.8.2.markdown b/notes/0.8.2.markdown
new file mode 100644
index 0000000..d62e8e6
--- /dev/null
+++ b/notes/0.8.2.markdown
@@ -0,0 +1,41 @@
+This is our first **binary** release as Apache Incubator.
+
+## Highlights
+
+- window DSL. Users can write window-aggregation  applications with high level DSL now.
+- experimental Redis connector.
+
+## Change log:
+
+- GEARPUMP-241 Update CHANGELOG.md to reflect all JIRA's 
+- GEARPUMP-240 Correct licenses
+- GEARPUMP-239 Correct binary and source licenses subdirectory
+- GEARPUMP-238 Correct release naming and packaging
+- GEARPUMP-237 do not manually download sbt launcher jar on travis
+- GEARPUMP-234 Update LICENSE and related files for 0.8.2-incubating binary release
+- GEARPUMP-230 Removed unused code from AppMasterService
+- GEARPUMP-227 remove unneeded license files
+- GEARPUMP-226 refactor build files
+- GEARPUMP-225 move partitioner from gearpump-core to module gearpump-streaming
+- GEARPUMP-224 Merge gearpump-daemon to gearpump-core
+- GEARPUMP-222 DFSJarStore is loaded even when local file path is configured
+- GEARPUMP-218 add shaded library as transitive dependencies
+- GEARPUMP-215 Gearpump Redis Integration - RedisStorage
+- GEARPUMP-213 build docs with MkDocs
+- GEARPUMP-212 Make Gearpump's shaded artifacts publishable
+- GEARPUMP-208 Demo site goes down a lot. Also references gearpump.io when connection is refused
+- GEARPUMP-207 remove hadoop dependency out of application path from dashboard
+- GEARPUMP-205 remove hdfs dependency from command gear's classpath
+- GEARPUMP-204 Add unit test for external_hbase module
+- GEARPUMP-203 Use DataSourceTask / DataSinkTask for DSL
+- GEARPUMP-201 integration test failure
+- GEARPUMP-197 Busy loop in FetchThread when incoming queue is full
+- GEARPUMP-189 Update project/Build.scala to enable publishing releases to https://repository.apache.org/content/repositories/releases/org/apache/gearpump/
+- GEARPUMP-188 use java.time.Instant for Task start time
+- GEARPUMP-185 Yarn kills Apache Gearpump's worker if the memory usage is too high
+- GEARPUMP-171 Fix some typos
+- GEARPUMP-48 Improve Java interoperability
+- GEARPUMP-37 Sometimes application clock is "not started" when the data stream is very slow
+- GEARPUMP-32 Minimum clock of source Tasks maybe inaccurate
+- GEARPUMP-23 add window DSL
+- GEARPUMP-15 Define initial Apache branded home website
\ No newline at end of file
diff --git a/project/Build.scala b/project/Build.scala
deleted file mode 100644
index a1e6ca5..0000000
--- a/project/Build.scala
+++ /dev/null
@@ -1,570 +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.
- */
-
-import com.typesafe.sbt.SbtPgp.autoImport._
-import BuildExample.examples
-import BuildIntegrationTest.integration_test
-import BuildShaded._
-import de.johoop.jacoco4sbt.JacocoPlugin.jacoco
-import org.scalajs.sbtplugin.ScalaJSPlugin.autoImport._
-import sbt.Keys._
-import sbt._
-import Pack.packProject
-import org.scalajs.sbtplugin.cross.CrossProject
-import sbtassembly.AssemblyPlugin.autoImport._
-import sbtunidoc.Plugin.UnidocKeys._
-import sbtunidoc.Plugin._
-import xerial.sbt.Sonatype._
-
-object Build extends sbt.Build {
-
-  val copySharedSourceFiles = TaskKey[Unit]("copied shared services source code")
-
-  val akkaVersion = "2.4.11"
-  val apacheRepo = "https://repository.apache.org/"
-  val hadoopVersion = "2.6.0"
-  val hbaseVersion = "1.0.0"
-  val commonsHttpVersion = "3.1"
-  val commonsLoggingVersion = "1.1.3"
-  val commonsLangVersion = "2.6"
-  val commonsIOVersion = "2.4"
-  val dataReplicationVersion = "0.7"
-  val upickleVersion = "0.3.4"
-  val junitVersion = "4.12"
-  val kafkaVersion = "0.8.2.1"
-  val stormVersion = "0.10.0"
-  val slf4jVersion = "1.7.7"
-
-  val crossScalaVersionNumbers = Seq("2.11.8")
-  val scalaVersionNumber = crossScalaVersionNumbers.last
-  val sprayVersion = "1.3.2"
-  val sprayJsonVersion = "1.3.1"
-  val scalaTestVersion = "2.2.0"
-  val scalaCheckVersion = "1.11.3"
-  val mockitoVersion = "1.10.17"
-  val bijectionVersion = "0.8.0"
-  val scalazVersion = "7.1.1"
-  val algebirdVersion = "0.9.0"
-  val chillVersion = "0.6.0"
-  val distDirectory = "output"
-  val projectName = "gearpump"
-
-  override def projects: Seq[Project] = (super.projects.toList ++ BuildExample.projects.toList
-    ++ Pack.projects.toList).toSeq
-
-  val commonSettings = Seq(jacoco.settings: _*) ++ sonatypeSettings ++
-    Seq(
-      resolvers ++= Seq(
-        "patriknw at bintray" at "http://dl.bintray.com/patriknw/maven",
-        "apache-repo" at "https://repository.apache.org/content/repositories",
-        "maven1-repo" at "http://repo1.maven.org/maven2",
-        "maven2-repo" at "http://mvnrepository.com/artifact",
-        "sonatype" at "https://oss.sonatype.org/content/repositories/releases",
-        "bintray/non" at "http://dl.bintray.com/non/maven",
-        "clockfly" at "http://dl.bintray.com/clockfly/maven",
-        "clojars" at "http://clojars.org/repo"
-      )
-      // ,addCompilerPlugin("org.scalamacros" % "paradise" % "2.1.0-M5" cross CrossVersion.full)
-    ) ++
-    Seq(
-      scalaVersion := scalaVersionNumber,
-      crossScalaVersions := crossScalaVersionNumbers,
-      organization := "org.apache.gearpump",
-      useGpg := false,
-      pgpSecretRing := file("./secring.asc"),
-      pgpPublicRing := file("./pubring.asc"),
-      scalacOptions ++= Seq("-Yclosure-elim", "-Yinline"),
-      publishMavenStyle := true,
-
-      pgpPassphrase := Option(System.getenv().get("PASSPHRASE")).map(_.toArray),
-      credentials += Credentials(
-        "Sonatype Nexus Repository Manager",
-        "repository.apache.org",
-        System.getenv().get("SONATYPE_USERNAME"),
-        System.getenv().get("SONATYPE_PASSWORD")),
-
-      pomIncludeRepository := { _ => false },
-
-      publishTo := {
-        if (isSnapshot.value) {
-          Some("snapshots" at apacheRepo + "content/repositories/snapshots")
-        } else {
-          Some("releases" at apacheRepo + "content/repositories/releases")
-        }
-      },
-
-      publishArtifact in Test := true,
-
-      pomExtra := {
-        <url>https://github.com/apache/incubator-gearpump</url>
-          <licenses>
-            <license>
-              <name>Apache 2</name>
-              <url>http://www.apache.org/licenses/LICENSE-2.0.txt</url>
-            </license>
-          </licenses>
-          <scm>
-            <connection>scm:git://git.apache.org/incubator-gearpump.git</connection>
-            <developerConnection>scm:git:git@github.com:apache/incubator-gearpump</developerConnection>
-            <url>github.com/apache/incubator-gearpump</url>
-          </scm>
-          <developers>
-            <developer>
-              <id>gearpump</id>
-              <name>Gearpump Team</name>
-              <url>http://gearpump.incubator.apache.org/community.html#who-we-are</url>
-            </developer>
-          </developers>
-      }
-    )
-
-  val noPublish = Seq(
-    publish := {},
-    publishLocal := {},
-    publishArtifact := false,
-    publishArtifact in Test := false
-  )
-
-  val daemonDependencies = Seq(
-    libraryDependencies ++= Seq(
-      "com.typesafe.akka" %% "akka-cluster" % akkaVersion,
-      "com.typesafe.akka" %% "akka-cluster-tools" % akkaVersion,
-      "commons-logging" % "commons-logging" % commonsLoggingVersion,
-      "com.typesafe.akka" %% "akka-distributed-data-experimental" % akkaVersion
-        exclude("com.typesafe.akka", "akka-stream_2.11"),
-      "org.apache.hadoop" % "hadoop-common" % hadoopVersion % "provided"
-    )
-  )
-
-  val coreDependencies = Seq(
-    libraryDependencies ++= Seq(
-      "org.slf4j" % "slf4j-api" % slf4jVersion,
-      "org.slf4j" % "slf4j-log4j12" % slf4jVersion,
-      "commons-lang" % "commons-lang" % commonsLangVersion,
-      "com.google.code.findbugs" % "jsr305" % "1.3.9" % "compile",
-
-      /**
-       * Overrides Netty version 3.10.3.Final used by Akka 2.4.2 to work-around netty hang issue
-       * (https://github.com/gearpump/gearpump/issues/2020)
-       *
-       * Akka 2.4.2 by default use Netty 3.10.3.Final, which has a serious issue which can hang
-       * the network. The same issue also happens in version range (3.10.0.Final, 3.10.5.Final)
-       * Netty 3.10.6.Final have this issue fixed, however, we find there is a 20% performance
-       * drop. So we decided to downgrade netty to 3.8.0.Final (Same version used in akka 2.3.12).
-       *
-       * @see https://github.com/gearpump/gearpump/pull/2017 for more discussions.
-       */
-      "io.netty" % "netty" % "3.8.0.Final",
-      "com.typesafe.akka" %% "akka-remote" % akkaVersion
-        exclude("io.netty", "netty"),
-
-      "com.typesafe.akka" %% "akka-actor" % akkaVersion,
-      "com.typesafe.akka" %% "akka-agent" % akkaVersion,
-      "com.typesafe.akka" %% "akka-slf4j" % akkaVersion,
-      "com.typesafe.akka" %% "akka-kernel" % akkaVersion,
-      "com.typesafe.akka" %% "akka-http-experimental" % akkaVersion,
-      "com.typesafe.akka" %% "akka-http-spray-json-experimental" % akkaVersion,
-      "org.scala-lang" % "scala-reflect" % scalaVersionNumber,
-      "org.scala-lang.modules" %% "scala-parser-combinators" % "1.0.4",
-      "com.typesafe.akka" %% "akka-testkit" % akkaVersion % "test"
-        exclude("com.typesafe.akka", "akka-stream_2.11"),
-      "org.scalatest" %% "scalatest" % scalaTestVersion % "test",
-      "org.scalacheck" %% "scalacheck" % scalaCheckVersion % "test",
-      "org.mockito" % "mockito-core" % mockitoVersion % "test",
-      "junit" % "junit" % junitVersion % "test"
-    ),
-
-    unmanagedJars in Compile ++= Seq(
-      getShadedJarFile(shaded_metrics_graphite.id, version.value),
-      getShadedJarFile(shaded_guava.id, version.value),
-      getShadedJarFile(shaded_akka_kryo.id, version.value)
-    )
-  )
-
-  lazy val javadocSettings = Seq(
-    addCompilerPlugin("com.typesafe.genjavadoc" %% "genjavadoc-plugin" %
-      "0.9" cross CrossVersion.full),
-    scalacOptions += s"-P:genjavadoc:out=${target.value}/java"
-  )
-
-  val myAssemblySettings = Seq(
-    test in assembly := {},
-    assemblyOption in assembly ~= {
-      _.copy(includeScala = false)
-    },
-    assemblyJarName in assembly := {
-      s"${name.value.split("-").last}-${scalaBinaryVersion.value}-${version.value}-assembly.jar"
-    }
-  )
-
-  val projectsWithDoc = inProjects(
-    core,
-    streaming,
-    external_kafka,
-    external_monoid,
-    external_serializer,
-    external_hbase,
-    external_hadoopfs)
-
-  lazy val gearpumpUnidocSetting = scalaJavaUnidocSettings ++ Seq(
-    unidocProjectFilter in(ScalaUnidoc, unidoc) := projectsWithDoc,
-    unidocProjectFilter in(JavaUnidoc, unidoc) := projectsWithDoc,
-
-    unidocAllSources in(ScalaUnidoc, unidoc) := {
-      ignoreUndocumentedPackages((unidocAllSources in(ScalaUnidoc, unidoc)).value)
-    },
-
-    // Skip class names containing $ and some internal packages in Javadocs
-    unidocAllSources in(JavaUnidoc, unidoc) := {
-      ignoreUndocumentedPackages((unidocAllSources in(JavaUnidoc, unidoc)).value)
-    }
-  )
-
-  private def ignoreUndocumentedPackages(packages: Seq[Seq[File]]): Seq[Seq[File]] = {
-    packages
-      .map(_.filterNot(_.getName.contains("$")))
-      .map(_.filterNot(_.getCanonicalPath.contains("akka")))
-  }
-
-  private def addShadedDeps(deps: Seq[xml.Node], node: xml.Node): xml.Node = {
-    node match {
-      case elem: xml.Elem =>
-        val child = if (elem.label == "dependencies") {
-          elem.child ++ deps
-        } else {
-          elem.child.map(addShadedDeps(deps, _))
-        }
-        xml.Elem(elem.prefix, elem.label, elem.attributes, elem.scope, false, child: _*)
-      case _ =>
-        node
-    }
-  }
-
-  lazy val root = Project(
-    id = "gearpump",
-    base = file("."),
-    settings = commonSettings ++ noPublish ++ gearpumpUnidocSetting)
-      .aggregate(shaded, core, daemon, streaming, services, external_kafka, external_monoid,
-      external_serializer, examples, akkastream, storm, yarn, external_hbase, packProject,
-      external_hadoopfs, integration_test).settings(Defaults.itSettings: _*)
-      .disablePlugins(sbtassembly.AssemblyPlugin)
-
-  lazy val core = Project(
-    id = "gearpump-core",
-    base = file("core"),
-    settings = commonSettings ++ javadocSettings ++ coreDependencies ++ Seq(
-      pomPostProcess := {
-        (node: xml.Node) => addShadedDeps(List(
-          getShadedDepXML(organization.value, shaded_akka_kryo.id, version.value),
-          getShadedDepXML(organization.value, shaded_guava.id, version.value),
-          getShadedDepXML(organization.value, shaded_metrics_graphite.id, version.value)), node)
-      }
-    ))
-      .disablePlugins(sbtassembly.AssemblyPlugin)
-
-  lazy val daemon = Project(
-    id = "gearpump-daemon",
-    base = file("daemon"),
-    settings = commonSettings ++ daemonDependencies)
-      .dependsOn(core % "test->test; compile->compile", cgroup % "test->test; compile->compile")
-      .disablePlugins(sbtassembly.AssemblyPlugin)
-
-  lazy val cgroup = Project(
-    id = "gearpump-experimental-cgroup",
-    base = file("experiments/cgroup"),
-    settings = commonSettings ++ noPublish ++ daemonDependencies)
-      .dependsOn (core % "test->test; compile->compile")
-      .disablePlugins(sbtassembly.AssemblyPlugin)
-
-  lazy val streaming = Project(
-    id = "gearpump-streaming",
-    base = file("streaming"),
-    settings = commonSettings ++ javadocSettings ++ Seq(
-      unmanagedJars in Compile ++= Seq(
-        getShadedJarFile(shaded_gs_collections.id, version.value)
-      ),
-
-      pomPostProcess := {
-        (node: xml.Node) => addShadedDeps(List(
-          getShadedDepXML(organization.value, shaded_gs_collections.id, version.value)), node)
-      }
-    ))
-    .dependsOn(core % "test->test; compile->compile", shaded_gs_collections, daemon % "test->test")
-    .disablePlugins(sbtassembly.AssemblyPlugin)
-
-  lazy val external_kafka = Project(
-    id = "gearpump-external-kafka",
-    base = file("external/kafka"),
-    settings = commonSettings ++ javadocSettings  ++
-      Seq(
-        libraryDependencies ++= Seq(
-          "org.apache.kafka" %% "kafka" % kafkaVersion,
-          "com.twitter" %% "bijection-core" % bijectionVersion,
-          ("org.apache.kafka" %% "kafka" % kafkaVersion classifier ("test")) % "test"
-        )
-      ))
-      .dependsOn (streaming % "test->test; provided")
-      .disablePlugins(sbtassembly.AssemblyPlugin)
-
-  lazy val services_full = CrossProject("gearpump-services", file("services"), CrossType.Full).
-    settings(
-      publish := {},
-      publishLocal := {}
-    ).disablePlugins(sbtassembly.AssemblyPlugin)
-
-  val distDashboardDirectory = s"${distDirectory}/target/pack/dashboard/views/scalajs"
-
-  // ScalaJs project need to be build seperately.
-  // sbt "project gearpump-servicesJS" compile
-  lazy val serviceJS: Project = services_full.js.settings(serviceJSSettings: _*)
-
-  lazy val services: Project = services_full.jvm.
-    settings(serviceJvmSettings: _*)
-    .settings(compile in Compile <<= (compile in Compile))
-    .dependsOn(streaming % "test->test;compile->compile")
-
-  lazy val serviceJvmSettings = commonSettings ++ noPublish ++ Seq(
-    libraryDependencies ++= Seq(
-      "com.typesafe.akka" %% "akka-http-testkit" % akkaVersion % "test"
-        exclude("com.typesafe.akka", "akka-stream_2.11"),
-      "org.scalatest" %% "scalatest" % scalaTestVersion % "test",
-      "com.lihaoyi" %% "upickle" % upickleVersion,
-      "com.softwaremill.akka-http-session" %% "core" % "0.2.5",
-      "com.typesafe.akka" %% "akka-http-spray-json-experimental" % akkaVersion
-        exclude("com.typesafe.akka", "akka-stream_2.11"),
-      "com.github.scribejava" % "scribejava-apis" % "2.4.0",
-      "com.ning" % "async-http-client" % "1.9.33",
-      "org.webjars" % "angularjs" % "1.4.9",
-
-      // angular 1.5 breaks ui-select, but we need ng-touch 1.5
-      "org.webjars.npm" % "angular-touch" % "1.5.0",
-      "org.webjars" % "angular-ui-router" % "0.2.15",
-      "org.webjars" % "bootstrap" % "3.3.6",
-      "org.webjars" % "d3js" % "3.5.6",
-      "org.webjars" % "momentjs" % "2.10.6",
-      "org.webjars" % "lodash" % "3.10.1",
-      "org.webjars" % "font-awesome" % "4.5.0",
-      "org.webjars" % "jquery" % "2.2.0",
-      "org.webjars" % "jquery-cookie" % "1.4.1",
-      "org.webjars.bower" % "angular-loading-bar" % "0.8.0",
-      "org.webjars.bower" % "angular-smart-table" % "2.1.6",
-      "org.webjars.bower" % "angular-motion" % "0.4.3",
-      "org.webjars.bower" % "bootstrap-additions" % "0.3.1",
-      "org.webjars.bower" % "angular-strap" % "2.3.5",
-      "org.webjars.npm" % "ui-select" % "0.14.2",
-      "org.webjars.bower" % "ng-file-upload" % "5.0.9",
-      "org.webjars.bower" % "vis" % "4.7.0",
-      "org.webjars.bower" % "clipboard.js" % "0.1.1",
-      "org.webjars.npm" % "dashing-deps" % "0.1.2",
-      "org.webjars.npm" % "dashing" % "0.4.8"
-    ).map(_.exclude("org.scalamacros", "quasiquotes_2.10"))
-      .map(_.exclude("org.scalamacros", "quasiquotes_2.10.3")))
-
-  lazy val serviceJSSettings = Seq(
-    scalaVersion := scalaVersionNumber,
-    crossScalaVersions := crossScalaVersionNumbers,
-    checksums := Seq(""),
-    requiresDOM := true,
-    libraryDependencies ++= Seq(
-      "com.lihaoyi" %%% "upickle" % upickleVersion,
-      "com.lihaoyi" %%% "utest" % "0.3.1"
-    ),
-    scalaJSStage in Global := FastOptStage,
-    testFrameworks += new TestFramework("utest.runner.Framework"),
-    requiresDOM := true,
-    persistLauncher in Compile := false,
-    persistLauncher in Test := false,
-    skip in packageJSDependencies := false,
-    scoverage.ScoverageSbtPlugin.ScoverageKeys.coverageExcludedPackages :=
-      ".*gearpump\\.dashboard.*",
-
-    copySharedSourceFiles := {
-      // scalastyle:off println
-      println(s"Copy shared source code to project services...")
-      // scalastyle:on println
-    },
-
-    artifactPath in fastOptJS in Compile :=
-      new java.io.File(distDashboardDirectory, moduleName.value + "-fastopt.js"),
-
-    fastOptJS in Compile <<= (fastOptJS in Compile).dependsOn(copySharedSourceFiles),
-
-    relativeSourceMaps := true,
-    jsEnv in Test := new PhantomJS2Env(scalaJSPhantomJSClassLoader.value))
-
-  lazy val akkastream = Project(
-    id = "gearpump-experiments-akkastream",
-    base = file("experiments/akkastream"),
-    settings = commonSettings ++ noPublish ++
-      Seq(
-        libraryDependencies ++= Seq(
-          "com.typesafe.akka" %% "akka-stream" % akkaVersion,
-          "org.apache.hadoop" % "hadoop-common" % hadoopVersion,
-          "org.json4s" %% "json4s-jackson" % "3.2.11",
-          "org.scalatest" %% "scalatest" % scalaTestVersion % "test"
-        )
-      ))
-      .dependsOn (services % "test->test; compile->compile", daemon % "test->test; compile->compile")
-      .disablePlugins(sbtassembly.AssemblyPlugin)
-
-  lazy val redis = Project(
-    id = "gearpump-experiments-redis",
-    base = file("experiments/redis"),
-    settings = commonSettings ++ noPublish ++ myAssemblySettings ++
-      Seq(
-        libraryDependencies ++= Seq(
-          "redis.clients" % "jedis" % "2.9.0"
-        ),
-        mainClass in(Compile, packageBin) := Some("org.apache.gearpump.example.Test")
-      ))
-      .dependsOn(streaming % "test->test; provided", daemon % "test->test; provided")
-
-  lazy val storm = Project(
-    id = "gearpump-experiments-storm",
-    base = file("experiments/storm"),
-    settings = commonSettings ++ noPublish ++
-      Seq(
-        libraryDependencies ++= Seq(
-          "commons-io" % "commons-io" % commonsIOVersion,
-          "org.apache.storm" % "storm-core" % stormVersion
-            exclude("clj-stacktrace", "clj-stacktrace")
-            exclude("ch.qos.logback", "logback-classic")
-            exclude("ch.qos.logback", "logback-core")
-            exclude("clj-time", "clj-time")
-            exclude("clout", "clout")
-            exclude("compojure", "compojure")
-            exclude("hiccup", "hiccup")
-            exclude("jline", "jline")
-            exclude("joda-time", "joda-time")
-            exclude("org.clojure", "core.incubator")
-            exclude("org.clojure", "math.numeric-tower")
-            exclude("org.clojure", "tools.logging")
-            exclude("org.clojure", "tools.cli")
-            exclude("org.clojure", "tools.macro")
-            exclude("org.mortbay.jetty", "jetty-util")
-            exclude("org.mortbay.jetty", "jetty")
-            exclude("org.ow2.asm", "asm")
-            exclude("org.slf4j", "log4j-over-slf4j")
-            exclude("org.apache.logging.log4j", "log4j-slf4j-impl")
-            exclude("ring", "ring-core")
-            exclude("ring", "ring-devel")
-            exclude("ring", "ring-jetty-adapter")
-            exclude("ring", "ring-servlet")
-        )
-      ))
-      .dependsOn (streaming % "test->test; compile->compile")
-      .disablePlugins(sbtassembly.AssemblyPlugin)
-
-  lazy val yarn = Project(
-    id = "gearpump-experiments-yarn",
-    base = file("experiments/yarn"),
-    settings = commonSettings ++ noPublish ++
-      Seq(
-        libraryDependencies ++= Seq(
-          "org.apache.hadoop" % "hadoop-hdfs" % hadoopVersion,
-          "org.apache.hadoop" % "hadoop-common" % hadoopVersion,
-          "org.apache.hadoop" % "hadoop-yarn-api" % hadoopVersion,
-          "org.apache.hadoop" % "hadoop-yarn-client" % hadoopVersion,
-          "org.apache.hadoop" % "hadoop-yarn-common" % hadoopVersion,
-          "commons-httpclient" % "commons-httpclient" % commonsHttpVersion,
-          "org.apache.hadoop" % "hadoop-mapreduce-client-core" % hadoopVersion,
-          "org.apache.hadoop" % "hadoop-yarn-server-resourcemanager" % hadoopVersion % "provided",
-          "org.apache.hadoop" % "hadoop-yarn-server-nodemanager" % hadoopVersion % "provided"
-        )
-      ))
-      .dependsOn(services % "test->test;compile->compile", daemon % "provided", core % "provided")
-      .disablePlugins(sbtassembly.AssemblyPlugin)
-
-  lazy val external_hbase = Project(
-    id = "gearpump-external-hbase",
-    base = file("external/hbase"),
-    settings = commonSettings ++ javadocSettings  ++
-      Seq(
-        libraryDependencies ++= Seq(
-          "org.apache.hadoop" % "hadoop-common" % hadoopVersion % "provided",
-          "org.apache.hadoop" % "hadoop-hdfs" % hadoopVersion % "provided",
-          "org.apache.hadoop" % "hadoop-mapreduce-client-core" % hadoopVersion % "provided",
-          "org.codehaus.jackson" % "jackson-core-asl" % "1.9.13" % "provided",
-          "org.codehaus.jackson" % "jackson-mapper-asl" % "1.9.13" % "provided",
-          "org.apache.hbase" % "hbase-client" % hbaseVersion
-            exclude("com.github.stephenc.findbugs", "findbugs-annotations")
-            exclude("com.google.guava", "guava")
-            exclude("commons-codec", "commons-codec")
-            exclude("commons-io", "commons-io")
-            exclude("commons-lang", "commons-lang")
-            exclude("commons-logging", "commons-logging")
-            exclude("io.netty", "netty")
-            exclude("junit", "junit")
-            exclude("log4j", "log4j")
-            exclude("org.apache.zookeeper", "zookeeper")
-            exclude("org.codehaus.jackson", "jackson-mapper-asl"),
-          "org.apache.hbase" % "hbase-client" % hbaseVersion,
-          "org.apache.hbase" % "hbase-common" % hbaseVersion
-            exclude("com.github.stephenc.findbugs", "findbugs-annotations")
-            exclude("com.google.guava", "guava")
-            exclude("commons-codec", "commons-codec")
-            exclude("commons-collections", "commons-collections")
-            exclude("commons-io", "commons-io")
-            exclude("commons-lang", "commons-lang")
-            exclude("commons-logging", "commons-logging")
-            exclude("junit", "junit")
-            exclude("log4j", "log4j")
-        )
-      ))
-      .dependsOn (streaming % "test->test; provided")
-      .disablePlugins(sbtassembly.AssemblyPlugin)
-
-  lazy val external_monoid = Project(
-    id = "gearpump-external-monoid",
-    base = file("external/monoid"),
-    settings = commonSettings ++ javadocSettings  ++
-      Seq(
-        libraryDependencies ++= Seq(
-          "com.twitter" %% "algebird-core" % algebirdVersion
-        )
-      ))
-      .dependsOn (streaming % "provided")
-      .disablePlugins(sbtassembly.AssemblyPlugin)
-
-  lazy val external_serializer = Project(
-    id = "gearpump-external-serializer",
-    base = file("external/serializer"),
-    settings = commonSettings ++ javadocSettings  ++
-      Seq(
-        libraryDependencies ++= Seq(
-          "com.twitter" %% "chill-bijection" % chillVersion
-            exclude("com.esotericsoftware.kryo", "kyro")
-            exclude("com.esotericsoftware.minlog", "minlog")
-        )
-      ))
-      .dependsOn (streaming % "provided")
-      .disablePlugins(sbtassembly.AssemblyPlugin)
-
-  lazy val external_hadoopfs = Project(
-    id = "gearpump-external-hadoopfs",
-    base = file("external/hadoopfs"),
-    settings = commonSettings ++ javadocSettings ++
-      Seq(
-        libraryDependencies ++= Seq(
-          "org.apache.hadoop" % "hadoop-common" % hadoopVersion % "provided",
-          "org.apache.hadoop" % "hadoop-hdfs" % hadoopVersion % "provided"
-        )
-      ))
-      .dependsOn (streaming % "test->test; provided")
-      .disablePlugins(sbtassembly.AssemblyPlugin)
-}
diff --git a/project/BuildDashboard.scala b/project/BuildDashboard.scala
new file mode 100644
index 0000000..cfa6aae
--- /dev/null
+++ b/project/BuildDashboard.scala
@@ -0,0 +1,119 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import BuildGearpump._
+import Dependencies._
+import org.scalajs.sbtplugin.cross.{CrossProject, CrossType}
+import org.scalajs.sbtplugin.ScalaJSPlugin.autoImport._
+import sbt._
+import sbt.Keys._
+
+object BuildDashboard extends sbt.Build {
+
+  lazy val services: Project = services_full.jvm
+    .settings(serviceJvmSettings: _*)
+    .dependsOn(core % "provided", streaming % "test->test; provided")
+
+  // ScalaJs project need to be build separately.
+  // sbt "project gearpump-servicesJS" compile
+  private lazy val serviceJS: Project = services_full.js.settings(serviceJSSettings: _*)
+
+  private lazy val services_full = CrossProject("gearpump-services", file("services"),
+    CrossType.Full)
+    .settings(
+      publish := {},
+      publishLocal := {}
+    ).disablePlugins(sbtassembly.AssemblyPlugin)
+
+  private val copySharedSourceFiles = TaskKey[Unit]("copied shared services source code")
+
+  private val distDashboardDirectory = s"${distDirectory}/target/pack/dashboard/views/scalajs"
+
+  private lazy val serviceJvmSettings = commonSettings ++ noPublish ++ Seq(
+    libraryDependencies ++= Seq(
+      "com.typesafe.akka" %% "akka-http-testkit" % akkaHttpVersion % "test",
+      "org.scalatest" %% "scalatest" % scalaTestVersion % "test",
+      "com.lihaoyi" %% "upickle" % upickleVersion,
+      "com.softwaremill.akka-http-session" %% "core" % "0.3.0",
+      "com.typesafe.akka" %% "akka-http-spray-json" % akkaHttpVersion,
+      "com.github.scribejava" % "scribejava-apis" % "2.4.0",
+      "com.ning" % "async-http-client" % "1.9.33",
+      "org.webjars" % "angularjs" % "1.4.9",
+
+      // angular 1.5 breaks ui-select, but we need ng-touch 1.5
+      "org.webjars.npm" % "angular-touch" % "1.5.0",
+      "org.webjars" % "angular-ui-router" % "0.2.15",
+      "org.webjars" % "bootstrap" % "3.3.6",
+      "org.webjars" % "d3js" % "3.5.6",
+      "org.webjars" % "momentjs" % "2.10.6",
+      "org.webjars" % "lodash" % "3.10.1",
+      "org.webjars" % "font-awesome" % "4.5.0",
+      "org.webjars" % "jquery" % "2.2.0",
+      "org.webjars" % "jquery-cookie" % "1.4.1",
+      "org.webjars.bower" % "angular-loading-bar" % "0.8.0"
+        exclude("org.webjars.bower", "angular"),
+      "org.webjars.bower" % "angular-smart-table" % "2.1.6"
+        exclude("org.webjars.bower", "angular"),
+      "org.webjars.bower" % "angular-motion" % "0.4.3",
+      "org.webjars.bower" % "bootstrap-additions" % "0.3.1",
+      "org.webjars.bower" % "angular-strap" % "2.3.5"
+        exclude("org.webjars.bower", "angular"),
+      "org.webjars.npm" % "ui-select" % "0.14.2",
+      "org.webjars.bower" % "ng-file-upload" % "5.0.9",
+      "org.webjars.bower" % "vis" % "4.7.0",
+      "org.webjars.bower" % "clipboard.js" % "0.1.1",
+      "org.webjars.npm" % "dashing-deps" % "0.1.2",
+      "org.webjars.npm" % "dashing" % "0.4.8"
+    ).map(_.exclude("org.scalamacros", "quasiquotes_2.10"))
+      .map(_.exclude("org.scalamacros", "quasiquotes_2.10.3")))
+
+  private lazy val serviceJSSettings = Seq(
+    scalaVersion := scalaVersionNumber,
+    crossScalaVersions := crossScalaVersionNumbers,
+    checksums := Seq(""),
+    requiresDOM := true,
+    libraryDependencies ++= Seq(
+      "com.lihaoyi" %%% "upickle" % upickleVersion,
+      "com.lihaoyi" %%% "utest" % "0.3.1"
+    ),
+    scalaJSStage in Global := FastOptStage,
+    testFrameworks += new TestFramework("utest.runner.Framework"),
+    requiresDOM := true,
+    persistLauncher in Compile := false,
+    persistLauncher in Test := false,
+    skip in packageJSDependencies := false,
+    scoverage.ScoverageSbtPlugin.ScoverageKeys.coverageExcludedPackages :=
+      ".*gearpump\\.dashboard.*",
+
+    copySharedSourceFiles := {
+      // scalastyle:off println
+      println(s"Copy shared source code to project services...")
+      // scalastyle:on println
+    },
+
+    artifactPath in fastOptJS in Compile :=
+      new java.io.File(distDashboardDirectory, moduleName.value + "-fastopt.js"),
+
+    fastOptJS in Compile := {
+      (fastOptJS in Compile).dependsOn(copySharedSourceFiles).value
+    },
+
+    relativeSourceMaps := true,
+    jsEnv in Test := new PhantomJS2Env(scalaJSPhantomJSClassLoader.value)
+  )
+}
\ No newline at end of file
diff --git a/project/BuildExample.scala b/project/BuildExample.scala
deleted file mode 100644
index 75fc9be..0000000
--- a/project/BuildExample.scala
+++ /dev/null
@@ -1,236 +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.
- */
-
-import sbt.Keys._
-import sbt._
-import Build._
-import sbtassembly.AssemblyPlugin.autoImport._
-
-object BuildExample extends sbt.Build {
-
-  lazy val examples = Project(
-    id = "gearpump-examples",
-    base = file("examples"),
-    settings = commonSettings ++ noPublish
-  ).aggregate(wordcount, wordcountJava, complexdag, sol, fsio, examples_kafka,
-    distributedshell, stockcrawler, transport, examples_state, pagerank, distributeservice).
-    disablePlugins(sbtassembly.AssemblyPlugin)
-
-  lazy val wordcountJava = Project(
-    id = "gearpump-examples-wordcountjava",
-    base = file("examples/streaming/wordcount-java"),
-    settings = commonSettings ++ noPublish ++ myAssemblySettings ++
-      Seq(
-        mainClass in(Compile, packageBin) :=
-          Some("org.apache.gearpump.streaming.examples.wordcountjava.WordCount"),
-
-        target in assembly := baseDirectory.value.getParentFile.getParentFile / "target" /
-          CrossVersion.binaryScalaVersion(scalaVersion.value)
-      )
-  ) dependsOn(streaming % "test->test; provided", daemon % "test->test; provided")
-
-  lazy val wordcount = Project(
-    id = "gearpump-examples-wordcount",
-    base = file("examples/streaming/wordcount"),
-    settings = commonSettings ++ noPublish ++ myAssemblySettings ++
-      Seq(
-        mainClass in(Compile, packageBin) :=
-          Some("org.apache.gearpump.streaming.examples.wordcount.WordCount"),
-
-        target in assembly := baseDirectory.value.getParentFile.getParentFile / "target" /
-          CrossVersion.binaryScalaVersion(scalaVersion.value)
-      )
-  ) dependsOn(streaming % "test->test; provided", daemon % "test->test; provided")
-
-  lazy val sol = Project(
-    id = "gearpump-examples-sol",
-    base = file("examples/streaming/sol"),
-    settings = commonSettings ++ noPublish ++ myAssemblySettings ++
-      Seq(
-        mainClass in(Compile, packageBin) := Some("org.apache.gearpump.streaming.examples.sol.SOL"),
-
-        target in assembly := baseDirectory.value.getParentFile.getParentFile / "target" /
-          CrossVersion.binaryScalaVersion(scalaVersion.value)
-
-      )
-  ) dependsOn (streaming % "test->test; provided")
-
-  lazy val complexdag = Project(
-    id = "gearpump-examples-complexdag",
-    base = file("examples/streaming/complexdag"),
-    settings = commonSettings ++ noPublish ++ myAssemblySettings ++
-      Seq(
-        mainClass in(Compile, packageBin) :=
-          Some("org.apache.gearpump.streaming.examples.complexdag.Dag"),
-
-        target in assembly := baseDirectory.value.getParentFile.getParentFile / "target" /
-          CrossVersion.binaryScalaVersion(scalaVersion.value)
-      )
-  ) dependsOn (streaming % "test->test; provided")
-
-  lazy val transport = Project(
-    id = "gearpump-examples-transport",
-    base = file("examples/streaming/transport"),
-    settings = commonSettings ++ noPublish ++ myAssemblySettings ++
-      Seq(
-        libraryDependencies ++= Seq(
-          "io.spray" %% "spray-can" % sprayVersion,
-          "io.spray" %% "spray-routing-shapeless2" % sprayVersion,
-          "io.spray" %% "spray-json" % sprayJsonVersion,
-          "com.lihaoyi" %% "upickle" % upickleVersion
-        ),
-        mainClass in(Compile, packageBin) :=
-          Some("org.apache.gearpump.streaming.examples.transport.Transport"),
-
-        target in assembly := baseDirectory.value.getParentFile.getParentFile / "target" /
-          CrossVersion.binaryScalaVersion(scalaVersion.value)
-      )
-  ) dependsOn (streaming % "test->test; provided")
-
-  lazy val distributedshell = Project(
-    id = "gearpump-examples-distributedshell",
-    base = file("examples/distributedshell"),
-    settings = commonSettings ++ noPublish ++ myAssemblySettings ++
-      Seq(
-        mainClass in(Compile, packageBin) :=
-          Some("org.apache.gearpump.examples.distributedshell.DistributedShell"),
-
-        target in assembly := baseDirectory.value.getParentFile / "target" /
-          CrossVersion.binaryScalaVersion(scalaVersion.value)
-      )
-  ) dependsOn (daemon % "test->test; provided")
-
-  lazy val distributeservice = Project(
-    id = "gearpump-examples-distributeservice",
-    base = file("examples/distributeservice"),
-    settings = commonSettings ++ noPublish ++ myAssemblySettings ++
-      Seq(
-        libraryDependencies ++= Seq(
-          "commons-httpclient" % "commons-httpclient" % commonsHttpVersion,
-          "commons-lang" % "commons-lang" % commonsLangVersion,
-          "commons-io" % "commons-io" % commonsIOVersion,
-          "io.spray" %% "spray-can" % sprayVersion,
-          "io.spray" %% "spray-routing-shapeless2" % sprayVersion
-        ),
-        mainClass in(Compile, packageBin) :=
-          Some("org.apache.gearpump.experiments.distributeservice.DistributeService"),
-
-        target in assembly := baseDirectory.value.getParentFile / "target" /
-          CrossVersion.binaryScalaVersion(scalaVersion.value)
-      )
-  ) dependsOn (daemon % "test->test; provided")
-
-  lazy val fsio = Project(
-    id = "gearpump-examples-fsio",
-    base = file("examples/streaming/fsio"),
-    settings = commonSettings ++ noPublish ++ myAssemblySettings ++
-      Seq(
-        libraryDependencies ++= Seq(
-          "org.apache.hadoop" % "hadoop-common" % hadoopVersion
-            exclude("org.mortbay.jetty", "jetty-util")
-            exclude("org.mortbay.jetty", "jetty")
-            exclude("org.fusesource.leveldbjni", "leveldbjni-all")
-            exclude("tomcat", "jasper-runtime")
-            exclude("commons-beanutils", "commons-beanutils-core")
-            exclude("commons-beanutils", "commons-beanutils")
-            exclude("asm", "asm")
-            exclude("org.ow2.asm", "asm")
-        ),
-        mainClass in(Compile, packageBin) :=
-          Some("org.apache.gearpump.streaming.examples.fsio.SequenceFileIO"),
-
-        target in assembly := baseDirectory.value.getParentFile.getParentFile / "target" /
-          CrossVersion.binaryScalaVersion(scalaVersion.value)
-      )
-  ) dependsOn (streaming % "test->test; provided")
-
-  lazy val examples_kafka = Project(
-    id = "gearpump-examples-kafka",
-    base = file("examples/streaming/kafka"),
-    settings = commonSettings ++ noPublish ++ myAssemblySettings ++
-      Seq(
-        mainClass in(Compile, packageBin) :=
-          Some("org.apache.gearpump.streaming.examples.kafka.wordcount.KafkaWordCount"),
-
-        target in assembly := baseDirectory.value.getParentFile.getParentFile / "target" /
-          CrossVersion.binaryScalaVersion(scalaVersion.value)
-      )
-  ) dependsOn(streaming % "test->test; provided", external_kafka)
-
-  lazy val stockcrawler = Project(
-    id = "gearpump-examples-stockcrawler",
-    base = file("examples/streaming/stockcrawler"),
-    settings = commonSettings ++ noPublish ++ myAssemblySettings ++
-      Seq(
-        libraryDependencies ++= Seq(
-          "io.spray" %% "spray-can" % sprayVersion,
-          "io.spray" %% "spray-routing-shapeless2" % sprayVersion,
-          "com.lihaoyi" %% "upickle" % upickleVersion,
-          "commons-httpclient" % "commons-httpclient" % commonsHttpVersion,
-          "net.sourceforge.htmlcleaner" % "htmlcleaner" % "2.2",
-          "joda-time" % "joda-time" % "2.7",
-          "org.joda" % "joda-convert" % "1.2",
-          "io.spray" %% "spray-json" % sprayJsonVersion
-        ),
-        mainClass in(Compile, packageBin) :=
-          Some("org.apache.gearpump.streaming.examples.stock.main.Stock"),
-
-        target in assembly := baseDirectory.value.getParentFile.getParentFile / "target" /
-          CrossVersion.binaryScalaVersion(scalaVersion.value)
-      )
-  ) dependsOn(streaming % "test->test; provided", external_kafka % "test->test")
-
-  lazy val examples_state = Project(
-    id = "gearpump-examples-state",
-    base = file("examples/streaming/state"),
-    settings = commonSettings ++ noPublish ++ myAssemblySettings ++
-      Seq(
-        libraryDependencies ++= Seq(
-          "org.apache.hadoop" % "hadoop-common" % hadoopVersion
-              exclude("org.mortbay.jetty", "jetty-util")
-              exclude("org.mortbay.jetty", "jetty")
-              exclude("org.fusesource.leveldbjni", "leveldbjni-all")
-              exclude("tomcat", "jasper-runtime")
-              exclude("commons-beanutils", "commons-beanutils-core")
-              exclude("commons-beanutils", "commons-beanutils")
-              exclude("asm", "asm")
-              exclude("org.ow2.asm", "asm"),
-          "org.apache.hadoop" % "hadoop-hdfs" % hadoopVersion
-        ),
-        mainClass in(Compile, packageBin) :=
-            Some("org.apache.gearpump.streaming.examples.state.MessageCountApp"),
-
-        target in assembly := baseDirectory.value.getParentFile.getParentFile / "target" /
-            CrossVersion.binaryScalaVersion(scalaVersion.value)
-      )
-  ).dependsOn(streaming % "test->test; provided",
-    external_hadoopfs, external_monoid, external_serializer, external_kafka)
-
-  lazy val pagerank = Project(
-    id = "gearpump-examples-pagerank",
-    base = file("examples/pagerank"),
-    settings = commonSettings ++ noPublish ++ myAssemblySettings ++
-      Seq(
-        mainClass in(Compile, packageBin) :=
-            Some("org.apache.gearpump.experiments.pagerank.example.PageRankExample"),
-
-        target in assembly := baseDirectory.value.getParentFile.getParentFile / "target" /
-            CrossVersion.binaryScalaVersion(scalaVersion.value)
-      )
-  ) dependsOn (streaming % "test->test; provided")
-}
diff --git a/project/BuildExamples.scala b/project/BuildExamples.scala
new file mode 100644
index 0000000..2af19ba
--- /dev/null
+++ b/project/BuildExamples.scala
@@ -0,0 +1,209 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import sbt.Keys._
+import sbt._
+import BuildGearpump._
+import BuildExternals._
+import Dependencies._
+import sbtassembly.AssemblyPlugin.autoImport._
+
+object BuildExamples extends sbt.Build {
+
+  lazy val examples: Seq[ProjectReference] = Seq(
+    complexdag,
+    distributedshell,
+    distributeservice,
+    examples_kafka,
+    examples_state,
+    fsio,
+    pagerank,
+    sol,
+    stockcrawler,
+    transport,
+    wordcount,
+    wordcountJava,
+    example_hbase
+  )
+
+  lazy val example_hbase = Project(
+    id = "gearpump-examples-hbase",
+    base = file("examples/streaming/hbase"),
+    settings = commonSettings ++ noPublish ++ myAssemblySettings ++
+      Seq(
+        libraryDependencies ++= Seq(
+          "org.apache.hadoop" % "hadoop-common" % hadoopVersion
+            exclude("commons-beanutils", "commons-beanutils-core")
+            exclude("commons-beanutils", "commons-beanutils")
+            exclude("asm", "asm")
+            exclude("org.ow2.asm", "asm")
+        ),
+        mainClass in(Compile, packageBin) :=
+          Some("org.apache.gearpump.streaming.examples.hbase.HBaseConn"),
+
+        target in assembly := baseDirectory.value.getParentFile.getParentFile / "target" /
+          CrossVersion.binaryScalaVersion(scalaVersion.value)
+      )
+  ) dependsOn(streaming % "test->test; provided", core % "provided", external_hbase)
+
+  lazy val wordcountJava = Project(
+    id = "gearpump-examples-wordcountjava",
+    base = file("examples/streaming/wordcount-java"),
+    settings = exampleSettings("org.apache.gearpump.streaming.examples.wordcountjava.WordCount")
+  ).dependsOn(core % "provided", streaming % "test->test; provided")
+
+  lazy val wordcount = Project(
+    id = "gearpump-examples-wordcount",
+    base = file("examples/streaming/wordcount"),
+    settings = exampleSettings("org.apache.gearpump.streaming.examples.wordcount.WordCount")
+  ).dependsOn(core % "provided", streaming % "test->test; provided")
+
+  lazy val sol = Project(
+    id = "gearpump-examples-sol",
+    base = file("examples/streaming/sol"),
+    settings = exampleSettings("org.apache.gearpump.streaming.examples.sol.SOL")
+  ).dependsOn(core % "provided", streaming % "test->test; provided")
+
+  lazy val complexdag = Project(
+    id = "gearpump-examples-complexdag",
+    base = file("examples/streaming/complexdag"),
+    settings = exampleSettings("org.apache.gearpump.streaming.examples.complexdag.Dag")
+  ).dependsOn(core % "provided", streaming % "test->test; provided")
+
+  lazy val transport = Project(
+    id = "gearpump-examples-transport",
+    base = file("examples/streaming/transport"),
+    settings = exampleSettings("org.apache.gearpump.streaming.examples.transport.Transport") ++
+      Seq(
+        libraryDependencies ++= Seq(
+          "io.spray" %% "spray-can" % sprayVersion,
+          "io.spray" %% "spray-routing-shapeless2" % sprayVersion,
+          "io.spray" %% "spray-json" % sprayJsonVersion,
+          "com.lihaoyi" %% "upickle" % upickleVersion
+        )
+      )
+  ).dependsOn(core % "provided", streaming % "test->test; provided")
+
+  lazy val distributedshell = Project(
+    id = "gearpump-examples-distributedshell",
+    base = file("examples/distributedshell"),
+    settings = commonSettings ++ noPublish ++ myAssemblySettings ++ Seq(
+      mainClass in(Compile, packageBin) :=
+        Some("org.apache.gearpump.examples.distributedshell.DistributedShell"),
+      target in assembly := baseDirectory.value.getParentFile / "target" /
+        CrossVersion.binaryScalaVersion(scalaVersion.value)
+    )
+  ).dependsOn(core % "test->test; provided")
+
+  lazy val distributeservice = Project(
+    id = "gearpump-examples-distributeservice",
+    base = file("examples/distributeservice"),
+    settings = commonSettings ++ noPublish ++ myAssemblySettings ++ Seq(
+      mainClass in(Compile, packageBin) :=
+        Some("org.apache.gearpump.experiments.distributeservice.DistributeService"),
+      target in assembly := baseDirectory.value.getParentFile / "target" /
+        CrossVersion.binaryScalaVersion(scalaVersion.value),
+      libraryDependencies ++= Seq(
+        "commons-httpclient" % "commons-httpclient" % commonsHttpVersion,
+        "commons-lang" % "commons-lang" % commonsLangVersion,
+        "commons-io" % "commons-io" % commonsIOVersion,
+        "io.spray" %% "spray-can" % sprayVersion,
+        "io.spray" %% "spray-routing-shapeless2" % sprayVersion
+        )
+    )
+  ).dependsOn(core % "test->test; provided")
+
+  lazy val fsio = Project(
+    id = "gearpump-examples-fsio",
+    base = file("examples/streaming/fsio"),
+    settings = exampleSettings("org.apache.gearpump.streaming.examples.fsio.SequenceFileIO") ++
+      Seq(
+        libraryDependencies ++= Seq(
+          "org.apache.hadoop" % "hadoop-common" % hadoopVersion
+            exclude("org.mortbay.jetty", "jetty-util")
+            exclude("org.mortbay.jetty", "jetty")
+            exclude("org.fusesource.leveldbjni", "leveldbjni-all")
+            exclude("tomcat", "jasper-runtime")
+            exclude("commons-beanutils", "commons-beanutils-core")
+            exclude("commons-beanutils", "commons-beanutils")
+            exclude("asm", "asm")
+            exclude("org.ow2.asm", "asm")
+        )
+      )
+  ).dependsOn(core % "provided", streaming % "test->test; provided")
+
+  lazy val examples_kafka = Project(
+    id = "gearpump-examples-kafka",
+    base = file("examples/streaming/kafka"),
+    settings =
+      exampleSettings("org.apache.gearpump.streaming.examples.kafka.wordcount.KafkaWordCount")
+  ).dependsOn(core % "provided", streaming % "test->test; provided", external_kafka)
+
+  lazy val stockcrawler = Project(
+    id = "gearpump-examples-stockcrawler",
+    base = file("examples/streaming/stockcrawler"),
+    settings = exampleSettings("org.apache.gearpump.streaming.examples.stock.main.Stock") ++
+      Seq(
+        libraryDependencies ++= Seq(
+          "io.spray" %% "spray-can" % sprayVersion,
+          "io.spray" %% "spray-routing-shapeless2" % sprayVersion,
+          "com.lihaoyi" %% "upickle" % upickleVersion,
+          "commons-httpclient" % "commons-httpclient" % commonsHttpVersion,
+          "net.sourceforge.htmlcleaner" % "htmlcleaner" % "2.2",
+          "joda-time" % "joda-time" % "2.7",
+          "org.joda" % "joda-convert" % "1.2",
+          "io.spray" %% "spray-json" % sprayJsonVersion
+        )
+      )
+  ).dependsOn(core % "provided", streaming % "test->test; provided", external_kafka % "test->test")
+
+  lazy val examples_state = Project(
+    id = "gearpump-examples-state",
+    base = file("examples/streaming/state"),
+    settings = exampleSettings("org.apache.gearpump.streaming.examples.state.MessageCountApp") ++
+      Seq(
+        libraryDependencies ++= Seq(
+          "org.apache.hadoop" % "hadoop-common" % hadoopVersion
+            exclude("org.mortbay.jetty", "jetty-util")
+            exclude("org.mortbay.jetty", "jetty")
+            exclude("org.fusesource.leveldbjni", "leveldbjni-all")
+            exclude("tomcat", "jasper-runtime")
+            exclude("commons-beanutils", "commons-beanutils-core")
+            exclude("commons-beanutils", "commons-beanutils")
+            exclude("asm", "asm")
+            exclude("org.ow2.asm", "asm"),
+          "org.apache.hadoop" % "hadoop-hdfs" % hadoopVersion
+        )
+      )
+  ).dependsOn(core % "provided", streaming % "test->test; provided",
+    external_hadoopfs, external_monoid, external_serializer, external_kafka)
+
+  lazy val pagerank = Project(
+    id = "gearpump-examples-pagerank",
+    base = file("examples/pagerank"),
+    settings = exampleSettings("org.apache.gearpump.experiments.pagerank.example.PageRankExample")
+  ).dependsOn(core % "provided", streaming % "test->test; provided")
+
+  private def exampleSettings(className: String): Seq[Def.Setting[_]] =
+    commonSettings ++ noPublish ++ myAssemblySettings ++ Seq(
+      mainClass in(Compile, packageBin) :=
+        Some(className),
+      target in assembly := baseDirectory.value.getParentFile.getParentFile / "target" /
+        CrossVersion.binaryScalaVersion(scalaVersion.value)
+    )
+}
diff --git a/project/BuildExperiments.scala b/project/BuildExperiments.scala
new file mode 100644
index 0000000..eb5f9e1
--- /dev/null
+++ b/project/BuildExperiments.scala
@@ -0,0 +1,119 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import BuildGearpump._
+import BuildDashboard.services
+import Dependencies._
+import sbt._
+import sbt.Keys._
+
+object BuildExperiments extends sbt.Build {
+
+  lazy val experiments: Seq[ProjectReference] = Seq(
+    akkastream,
+    cgroup,
+    redis,
+    storm,
+    yarn
+  )
+
+  lazy val yarn = Project(
+    id = "gearpump-experiments-yarn",
+    base = file("experiments/yarn"),
+    settings = commonSettings ++ noPublish ++
+      Seq(
+        libraryDependencies ++= Seq(
+          "org.apache.hadoop" % "hadoop-yarn-api" % hadoopVersion,
+          "org.apache.hadoop" % "hadoop-yarn-client" % hadoopVersion,
+          "org.apache.hadoop" % "hadoop-yarn-common" % hadoopVersion,
+          "commons-httpclient" % "commons-httpclient" % commonsHttpVersion,
+          "org.apache.hadoop" % "hadoop-mapreduce-client-core" % hadoopVersion,
+          "org.apache.hadoop" % "hadoop-yarn-server-resourcemanager" % hadoopVersion % "provided",
+          "org.apache.hadoop" % "hadoop-yarn-server-nodemanager" % hadoopVersion % "provided"
+        )
+      ))
+    .dependsOn(services % "test->test;compile->compile",
+      core % "provided", gearpumpHadoop).disablePlugins(sbtassembly.AssemblyPlugin)
+
+  lazy val akkastream = Project(
+    id = "gearpump-experiments-akkastream",
+    base = file("experiments/akkastream"),
+    settings = commonSettings ++ noPublish ++ myAssemblySettings ++
+      Seq(
+        libraryDependencies ++= Seq(
+          "org.json4s" %% "json4s-jackson" % "3.2.11"
+        ),
+        mainClass in(Compile, packageBin) := Some("akka.stream.gearpump.example.Test")
+      ))
+    .dependsOn(core % "provided", streaming % "test->test; provided")
+
+  lazy val redis = Project(
+    id = "gearpump-experiments-redis",
+    base = file("experiments/redis"),
+    settings = commonSettings ++ noPublish ++
+      Seq(
+        libraryDependencies ++= Seq(
+          "redis.clients" % "jedis" % jedisVersion
+        )
+      )
+  ).dependsOn(core % "provided", streaming % "test->test; provided")
+
+  lazy val storm = Project(
+    id = "gearpump-experiments-storm",
+    base = file("experiments/storm"),
+    settings = commonSettings ++ noPublish ++
+      Seq(
+        libraryDependencies ++= Seq(
+          "commons-io" % "commons-io" % commonsIOVersion,
+          "org.apache.storm" % "storm-core" % stormVersion
+            exclude("clj-stacktrace", "clj-stacktrace")
+            exclude("ch.qos.logback", "logback-classic")
+            exclude("ch.qos.logback", "logback-core")
+            exclude("clj-time", "clj-time")
+            exclude("clout", "clout")
+            exclude("compojure", "compojure")
+            exclude("hiccup", "hiccup")
+            exclude("jline", "jline")
+            exclude("joda-time", "joda-time")
+            exclude("org.clojure", "core.incubator")
+            exclude("org.clojure", "math.numeric-tower")
+            exclude("org.clojure", "tools.logging")
+            exclude("org.clojure", "tools.cli")
+            exclude("org.clojure", "tools.macro")
+            exclude("org.mortbay.jetty", "jetty-util")
+            exclude("org.mortbay.jetty", "jetty")
+            exclude("org.ow2.asm", "asm")
+            exclude("org.slf4j", "log4j-over-slf4j")
+            exclude("org.slf4j", "slf4j-api")
+            exclude("org.apache.logging.log4j", "log4j-slf4j-impl")
+            exclude("ring", "ring-core")
+            exclude("ring", "ring-devel")
+            exclude("ring", "ring-jetty-adapter")
+            exclude("ring", "ring-servlet")
+        )
+      ))
+    .dependsOn (core % "provided", streaming % "test->test; provided")
+    .disablePlugins(sbtassembly.AssemblyPlugin)
+
+  lazy val cgroup = Project(
+    id = "gearpump-experimental-cgroup",
+    base = file("experiments/cgroup"),
+    settings = commonSettings ++ noPublish)
+    .dependsOn (core % "provided")
+    .disablePlugins(sbtassembly.AssemblyPlugin)
+}
\ No newline at end of file
diff --git a/project/BuildExternals.scala b/project/BuildExternals.scala
new file mode 100644
index 0000000..48715cb
--- /dev/null
+++ b/project/BuildExternals.scala
@@ -0,0 +1,124 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import BuildGearpump._
+import Dependencies._
+import Docs._
+import sbt._
+import sbt.Keys._
+
+object BuildExternals extends sbt.Build {
+
+  lazy val externals: Seq[ProjectReference] = Seq(
+    external_hbase,
+    external_kafka,
+    external_monoid,
+    external_hadoopfs
+  )
+
+  lazy val external_kafka = Project(
+    id = "gearpump-external-kafka",
+    base = file("external/kafka"),
+    settings = commonSettings ++ javadocSettings  ++
+      Seq(
+        libraryDependencies ++= Seq(
+          "org.apache.kafka" %% "kafka" % kafkaVersion,
+          "com.twitter" %% "bijection-core" % bijectionVersion,
+          ("org.apache.kafka" %% "kafka" % kafkaVersion classifier ("test")) % "test"
+        )
+      ))
+    .dependsOn(core % "provided", streaming % "test->test; provided")
+    .disablePlugins(sbtassembly.AssemblyPlugin)
+
+  lazy val external_hbase = Project(
+    id = "gearpump-external-hbase",
+    base = file("external/hbase"),
+    settings = commonSettings ++ javadocSettings  ++
+      Seq(
+        libraryDependencies ++= Seq(
+          "org.apache.hadoop" % "hadoop-common" % hadoopVersion % "provided",
+          "org.apache.hadoop" % "hadoop-hdfs" % hadoopVersion % "provided",
+          "org.apache.hadoop" % "hadoop-mapreduce-client-core" % hadoopVersion % "provided",
+          "org.codehaus.jackson" % "jackson-core-asl" % "1.9.13" % "provided",
+          "org.codehaus.jackson" % "jackson-mapper-asl" % "1.9.13" % "provided",
+          "org.apache.hbase" % "hbase-client" % hbaseVersion
+            exclude("com.github.stephenc.findbugs", "findbugs-annotations")
+            exclude("com.google.guava", "guava")
+            exclude("commons-codec", "commons-codec")
+            exclude("commons-io", "commons-io")
+            exclude("commons-lang", "commons-lang")
+            exclude("commons-logging", "commons-logging")
+            exclude("io.netty", "netty")
+            exclude("junit", "junit")
+            exclude("log4j", "log4j")
+            exclude("org.apache.zookeeper", "zookeeper")
+            exclude("org.codehaus.jackson", "jackson-mapper-asl"),
+          "org.apache.hbase" % "hbase-common" % hbaseVersion
+            exclude("com.github.stephenc.findbugs", "findbugs-annotations")
+            exclude("com.google.guava", "guava")
+            exclude("commons-codec", "commons-codec")
+            exclude("commons-collections", "commons-collections")
+            exclude("commons-io", "commons-io")
+            exclude("commons-lang", "commons-lang")
+            exclude("commons-logging", "commons-logging")
+            exclude("junit", "junit")
+            exclude("log4j", "log4j")
+        )
+      ))
+    .dependsOn (core % "provided", streaming % "test->test; provided")
+    .disablePlugins(sbtassembly.AssemblyPlugin)
+
+  lazy val external_monoid = Project(
+    id = "gearpump-external-monoid",
+    base = file("external/monoid"),
+    settings = commonSettings ++ javadocSettings  ++
+      Seq(
+        libraryDependencies ++= Seq(
+          "com.twitter" %% "algebird-core" % algebirdVersion
+        )
+      ))
+    .dependsOn (core % "provided", streaming % "provided")
+    .disablePlugins(sbtassembly.AssemblyPlugin)
+
+  lazy val external_serializer = Project(
+    id = "gearpump-external-serializer",
+    base = file("external/serializer"),
+    settings = commonSettings ++ javadocSettings  ++
+      Seq(
+        libraryDependencies ++= Seq(
+          "com.twitter" %% "chill-bijection" % chillVersion
+            exclude("com.esotericsoftware.kryo", "kyro")
+            exclude("com.esotericsoftware.minlog", "minlog")
+        )
+      ))
+    .dependsOn (core % "provided", streaming % "provided")
+    .disablePlugins(sbtassembly.AssemblyPlugin)
+
+  lazy val external_hadoopfs = Project(
+    id = "gearpump-external-hadoopfs",
+    base = file("external/hadoopfs"),
+    settings = commonSettings ++ javadocSettings ++
+      Seq(
+        libraryDependencies ++= Seq(
+          "org.apache.hadoop" % "hadoop-common" % hadoopVersion % "provided",
+          "org.apache.hadoop" % "hadoop-hdfs" % hadoopVersion % "provided"
+        )
+      ))
+    .dependsOn(core % "provided", streaming % "test->test; provided")
+    .disablePlugins(sbtassembly.AssemblyPlugin)
+}
\ No newline at end of file
diff --git a/project/BuildGearpump.scala b/project/BuildGearpump.scala
new file mode 100644
index 0000000..f0c9517
--- /dev/null
+++ b/project/BuildGearpump.scala
@@ -0,0 +1,253 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import com.typesafe.sbt.SbtPgp.autoImport._
+import BuildExamples.examples
+import BuildExperiments.experiments
+import BuildExternals.externals
+import BuildIntegrationTests.integrationTests
+import BuildDashboard.services
+import Dependencies._
+import Docs._
+import de.johoop.jacoco4sbt.JacocoPlugin.jacoco
+import sbt.Keys._
+import sbt._
+import Pack.packProject
+import sbtassembly.AssemblyPlugin.autoImport._
+
+import xerial.sbt.Sonatype._
+
+object BuildGearpump extends sbt.Build {
+
+  val apacheRepo = "https://repository.apache.org/"
+  val distDirectory = "output"
+  val projectName = "gearpump"
+
+  val commonSettings = Seq(jacoco.settings: _*) ++ sonatypeSettings ++
+    Seq(
+      resolvers ++= Seq(
+        // https://repo1.maven.org/maven2 has been added by default
+        "apache-repo" at "https://repository.apache.org/content/repositories",
+        Resolver.sonatypeRepo("releases"),
+        "clojars" at "http://clojars.org/repo"
+      )
+    ) ++
+    Seq(
+      scalaVersion := scalaVersionNumber,
+      crossScalaVersions := crossScalaVersionNumbers,
+      organization := "org.apache.gearpump",
+      useGpg := false,
+      pgpSecretRing := file("./secring.asc"),
+      pgpPublicRing := file("./pubring.asc"),
+      scalacOptions ++= Seq("-Yclosure-elim", "-Yinline"),
+      publishMavenStyle := true,
+
+      pgpPassphrase := Option(System.getenv().get("PASSPHRASE")).map(_.toArray),
+      credentials += Credentials(
+        "Sonatype Nexus Repository Manager",
+        "repository.apache.org",
+        System.getenv().get("SONATYPE_USERNAME"),
+        System.getenv().get("SONATYPE_PASSWORD")),
+
+      pomIncludeRepository := { _ => false },
+
+      publishTo := {
+        if (isSnapshot.value) {
+          Some("snapshots" at apacheRepo + "content/repositories/snapshots")
+        } else {
+          Some("releases" at apacheRepo + "content/repositories/releases")
+        }
+      },
+
+      publishArtifact in Test := true,
+
+      pomExtra := {
+        <url>https://github.com/apache/incubator-gearpump</url>
+          <licenses>
+            <license>
+              <name>Apache 2</name>
+              <url>http://www.apache.org/licenses/LICENSE-2.0.txt</url>
+            </license>
+          </licenses>
+          <scm>
+            <connection>scm:git://git.apache.org/incubator-gearpump.git</connection>
+            <developerConnection>scm:git:git@github.com:apache/incubator-gearpump</developerConnection>
+            <url>github.com/apache/incubator-gearpump</url>
+          </scm>
+          <developers>
+            <developer>
+              <id>gearpump</id>
+              <name>Gearpump Team</name>
+              <url>http://gearpump.incubator.apache.org/community.html#who-we-are</url>
+            </developer>
+          </developers>
+      },
+
+      pomPostProcess := {
+        (node: xml.Node) => changeShadedDeps(
+          Set(
+            "org.scoverage",
+            "org.scala-lang"
+          ),
+          List.empty[xml.Node],
+          node)
+      },
+
+      cleanFiles += (baseDirectory.value / "examples" / "target")
+    )
+
+  val noPublish = Seq(
+    publish := {},
+    publishLocal := {},
+    publishArtifact := false,
+    publishArtifact in Test := false
+  )
+
+  lazy val myAssemblySettings = Seq(
+    test in assembly := {},
+    assemblyOption in assembly ~= {
+      _.copy(includeScala = false)
+    },
+    assemblyJarName in assembly := {
+      s"${name.value}_${scalaBinaryVersion.value}-${version.value}.jar"
+    },
+    assemblyShadeRules in assembly := Seq(
+      ShadeRule.rename("com.romix.**" -> "org.apache.gearpump.@0").inAll,
+      ShadeRule.rename("com.esotericsoftware.**" ->
+        "org.apache.gearpump.@0").inAll,
+      ShadeRule.rename("org.objenesis.**" -> "org.apache.gearpump.@0").inAll,
+      ShadeRule.rename("com.google.common.**" -> "org.apache.gearpump.@0").inAll,
+      ShadeRule.rename("com.google.thirdparty.**" -> "org.apache.gearpump.@0").inAll,
+      ShadeRule.rename("com.codahale.metrics.**" ->
+        "org.apache.gearpump.@0").inAll,
+      ShadeRule.rename("com.gs.collections.**" ->
+        "org.apache.gearpump.gs.collections.@0").inAll
+    ),
+    target in assembly := baseDirectory.value / "target" / scalaBinaryVersion.value
+  )
+
+  lazy val aggregated: Seq[ProjectReference] = Seq[ProjectReference](
+    core,
+    streaming,
+    services,
+    gearpumpHadoop,
+    packProject
+  ) ++ examples ++ experiments ++ externals ++ integrationTests
+
+  lazy val root = Project(
+    id = "gearpump",
+    base = file("."),
+    settings = commonSettings ++ noPublish ++ gearpumpUnidocSetting,
+    aggregate = aggregated)
+    .settings(Defaults.itSettings: _*)
+    .disablePlugins(sbtassembly.AssemblyPlugin)
+
+  lazy val core = Project(
+    id = "gearpump-core",
+    base = file("core"),
+    settings = commonSettings ++ myAssemblySettings ++ javadocSettings ++ coreDependencies ++
+      addArtifact(Artifact("gearpump-core"), sbtassembly.AssemblyKeys.assembly) ++ Seq(
+
+      assemblyOption in assembly ~= {
+        _.copy(includeScala = true)
+      },
+
+      pomPostProcess := {
+        (node: xml.Node) => changeShadedDeps(
+          Set(
+            "com.github.romix.akka",
+            "com.google.guava",
+            "com.codahale.metrics",
+            "org.scoverage"
+          ), List.empty[xml.Node], node)
+      }
+    ))
+
+  lazy val streaming = Project(
+    id = "gearpump-streaming",
+    base = file("streaming"),
+    settings = commonSettings ++ myAssemblySettings ++ javadocSettings ++
+      addArtifact(Artifact("gearpump-streaming"), sbtassembly.AssemblyKeys.assembly) ++
+      Seq(
+        assemblyMergeStrategy in assembly := {
+          case "geardefault.conf" =>
+            MergeStrategy.last
+          case x =>
+            val oldStrategy = (assemblyMergeStrategy in assembly).value
+            oldStrategy(x)
+        },
+
+        libraryDependencies ++= Seq(
+          "com.goldmansachs" % "gs-collections" % gsCollectionsVersion
+        ),
+
+        pomPostProcess := {
+          (node: xml.Node) => changeShadedDeps(
+            Set(
+              "com.goldmansachs",
+              "org.scala-lang",
+              "org.scoverage"
+            ),
+            List(
+              getShadedDepXML(organization.value, s"${core.id}_${scalaBinaryVersion.value}",
+                version.value, "provided")),
+            node)
+        }
+      )
+  ).dependsOn(core % "test->test;provided")
+
+  lazy val gearpumpHadoop = Project(
+    id = "gearpump-hadoop",
+    base = file("gearpump-hadoop"),
+    settings = commonSettings ++ noPublish ++
+      Seq(
+        libraryDependencies ++= Seq(
+          "org.apache.hadoop" % "hadoop-hdfs" % hadoopVersion,
+          "org.apache.hadoop" % "hadoop-common" % hadoopVersion
+        )
+      )
+  ).dependsOn(core % "provided").disablePlugins(sbtassembly.AssemblyPlugin)
+
+  private def changeShadedDeps(toExclude: Set[String], toInclude: List[xml.Node],
+      node: xml.Node): xml.Node = {
+    node match {
+      case elem: xml.Elem =>
+        val child =
+          if (elem.label == "dependencies") {
+            elem.child.filterNot { dep =>
+              dep.child.find(_.label == "groupId").exists(gid => toExclude.contains(gid.text))
+            } ++ toInclude
+          } else {
+            elem.child.map(changeShadedDeps(toExclude, toInclude, _))
+          }
+        xml.Elem(elem.prefix, elem.label, elem.attributes, elem.scope, false, child: _*)
+      case _ =>
+        node
+    }
+  }
+
+  private def getShadedDepXML(groupId: String, artifactId: String,
+      version: String, scope: String): scala.xml.Node = {
+    <dependency>
+      <groupId>{groupId}</groupId>
+      <artifactId>{artifactId}</artifactId>
+      <version>{version}</version>
+      <scope>{scope}</scope>
+    </dependency>
+  }
+}
diff --git a/project/BuildIntegrationTest.scala b/project/BuildIntegrationTests.scala
similarity index 83%
rename from project/BuildIntegrationTest.scala
rename to project/BuildIntegrationTests.scala
index 6eed7a2..161d906 100644
--- a/project/BuildIntegrationTest.scala
+++ b/project/BuildIntegrationTests.scala
@@ -18,27 +18,26 @@
 
 import sbt.Keys._
 import sbt._
-import Build._
+import BuildGearpump._
+import BuildExperiments.storm
+import BuildDashboard.services
+import BuildExternals.{external_kafka, external_serializer}
+import Dependencies._
 import sbtassembly.AssemblyPlugin.autoImport._
 
-object BuildIntegrationTest extends sbt.Build {
+object BuildIntegrationTests extends sbt.Build {
 
-  val jsonSimpleVersion = "1.1"
-  val storm09Version = "0.9.6"
+  lazy val integrationTests: Seq[ProjectReference] = Seq(
+    it_core,
+    it_storm09,
+    it_storm010
+  )
 
-  lazy val integration_test = Project(
-    id = "gearpump-integrationtest",
-    base = file("integrationtest"),
-    settings = commonSettings ++ noPublish
-  ).aggregate(it_core, it_storm09, it_storm010).
-    disablePlugins(sbtassembly.AssemblyPlugin)
-
-  val itTestFilter: String => Boolean = { name => name endsWith "Suite" }
   lazy val it_core = Project(
     id = "gearpump-integrationtest-core",
     base = file("integrationtest/core"),
     settings = commonSettings ++ noPublish ++ Seq(
-      testOptions in IntegrationTest += Tests.Filter(itTestFilter),
+      testOptions in IntegrationTest += Tests.Filter(_.endsWith("Suite")),
       libraryDependencies ++= Seq(
         "com.lihaoyi" %% "upickle" % upickleVersion,
         "org.scalatest" %% "scalatest" % scalaTestVersion % "it",
@@ -51,6 +50,7 @@
     )
   ).configs(IntegrationTest).settings(Defaults.itSettings: _*)
     .dependsOn(
+      core % "provided",
       streaming % "test->test; provided",
       services % "test->test; provided",
       external_kafka,
@@ -73,7 +73,7 @@
         target in assembly := baseDirectory.value.getParentFile / "target" /
           CrossVersion.binaryScalaVersion(scalaVersion.value)
       )
-  ) dependsOn (storm % "provided")
+  ).dependsOn(core % "provided", storm % "provided")
 
   // Integration test for Storm 0.10.x
   lazy val it_storm010 = Project(
@@ -89,5 +89,5 @@
         target in assembly := baseDirectory.value.getParentFile / "target" /
           CrossVersion.binaryScalaVersion(scalaVersion.value)
       )
-  ) dependsOn (storm % "provided")
+  ).dependsOn(core % "provided", storm % "provided")
 }
diff --git a/project/BuildShaded.scala b/project/BuildShaded.scala
deleted file mode 100644
index a43587c..0000000
--- a/project/BuildShaded.scala
+++ /dev/null
@@ -1,142 +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.
- */
-
-import sbt.Keys._
-import sbt._
-import sbtassembly.AssemblyPlugin.autoImport._
-
-object BuildShaded extends sbt.Build {
-
-  val guavaVersion = "16.0.1"
-  val codahaleVersion = "3.0.2"
-  val kryoVersion = "0.4.1"
-  val gsCollectionsVersion = "6.2.0"
-  private val scalaVersionMajor = "2.11"
-
-  val shadeAssemblySettings = Build.commonSettings ++ Seq(
-    scalaVersion := Build.scalaVersionNumber,
-    test in assembly := {},
-    assemblyOption in assembly ~= {
-      _.copy(includeScala = false)
-    },
-    assemblyJarName in assembly := {
-      s"${name.value}_$scalaVersionMajor-${version.value}.jar"
-    },
-    target in assembly := baseDirectory.value.getParentFile / "target" / scalaVersionMajor
-  )
-
-  val shaded = Project(
-    id = "gearpump-shaded",
-    base = file("shaded")
-  ).aggregate(shaded_akka_kryo, shaded_gs_collections, shaded_guava, shaded_metrics_graphite)
-    .disablePlugins(sbtassembly.AssemblyPlugin)
-
-  lazy val shaded_akka_kryo = Project(
-    id = "gearpump-shaded-akka-kryo",
-    base = file("shaded/akka-kryo"),
-    settings = shadeAssemblySettings ++ addArtifact(Artifact("gearpump-shaded-akka-kryo"),
-      sbtassembly.AssemblyKeys.assembly) ++
-      Seq(
-        assemblyShadeRules in assembly := Seq(
-          ShadeRule.zap("com.google.protobuf.**").inAll,
-          ShadeRule.zap("com.typesafe.config.**").inAll,
-          ShadeRule.zap("akka.**").inAll,
-          ShadeRule.zap("org.jboss.netty.**").inAll,
-          ShadeRule.zap("net.jpountz.lz4.**").inAll,
-          ShadeRule.zap("org.uncommons.maths.**").inAll,
-          ShadeRule.rename("com.romix.**" -> "org.apache.gearpump.romix.@1").inAll,
-          ShadeRule.rename("com.esotericsoftware.**" ->
-            "org.apache.gearpump.esotericsoftware.@1").inAll,
-          ShadeRule.rename("org.objenesis.**" -> "org.apache.gearpump.objenesis.@1").inAll
-        )
-      ) ++
-      Seq(
-        libraryDependencies ++= Seq(
-          "com.github.romix.akka" %% "akka-kryo-serialization" % kryoVersion
-        )
-      )
-  )
-
-  lazy val shaded_gs_collections = Project(
-    id = "gearpump-shaded-gs-collections",
-    base = file("shaded/gs-collections"),
-    settings = shadeAssemblySettings ++ addArtifact(Artifact("gearpump-shaded-gs-collections"),
-      sbtassembly.AssemblyKeys.assembly) ++
-      Seq(
-        assemblyShadeRules in assembly := Seq(
-          ShadeRule.rename("com.gs.collections.**" ->
-            "org.apache.gearpump.gs.collections.@1").inAll
-        )
-      ) ++
-      Seq(
-        libraryDependencies ++= Seq(
-          "com.goldmansachs" % "gs-collections" % gsCollectionsVersion
-        )
-      )
-  )
-
-  lazy val shaded_guava = Project(
-    id = "gearpump-shaded-guava",
-    base = file("shaded/guava"),
-    settings = shadeAssemblySettings ++ addArtifact(Artifact("gearpump-shaded-guava"),
-      sbtassembly.AssemblyKeys.assembly) ++
-      Seq(
-        assemblyShadeRules in assembly := Seq(
-          ShadeRule.rename("com.google.**" -> "org.apache.gearpump.google.@1").inAll
-        )
-      ) ++
-      Seq(
-        libraryDependencies ++= Seq(
-          "com.google.guava" % "guava" % guavaVersion
-        )
-      )
-  )
-
-  lazy val shaded_metrics_graphite = Project(
-    id = "gearpump-shaded-metrics-graphite",
-    base = file("shaded/metrics-graphite"),
-    settings = shadeAssemblySettings ++ addArtifact(Artifact("gearpump-shaded-metrics-graphite"),
-      sbtassembly.AssemblyKeys.assembly) ++
-      Seq(
-        assemblyShadeRules in assembly := Seq(
-          ShadeRule.rename("com.codahale.metrics.**" ->
-            "org.apache.gearpump.codahale.metrics.@1").inAll
-        )
-      ) ++
-      Seq(
-        libraryDependencies ++= Seq(
-          "com.codahale.metrics" % "metrics-graphite" % codahaleVersion,
-          "com.codahale.metrics" % "metrics-jvm" % codahaleVersion
-        )
-      )
-  )
-
-  def getShadedJarFile(name: String, gearpumpVersion: String): File = {
-    shaded.base / "target" / scalaVersionMajor /
-      s"${name}_$scalaVersionMajor-$gearpumpVersion.jar"
-  }
-
-  def getShadedDepXML(groupId: String, artifactId: String, version: String): scala.xml.Node = {
-    <dependency>
-      <groupId>{groupId}</groupId>
-      <artifactId>{artifactId}</artifactId>
-      <version>{version}</version>
-    </dependency>
-  }
-
-}
\ No newline at end of file
diff --git a/project/Dependencies.scala b/project/Dependencies.scala
new file mode 100644
index 0000000..4e30d3f
--- /dev/null
+++ b/project/Dependencies.scala
@@ -0,0 +1,102 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import sbt._
+import Keys._
+
+object Dependencies {
+
+  val crossScalaVersionNumbers = Seq("2.11.8")
+  val scalaVersionNumber = crossScalaVersionNumbers.last
+  val akkaVersion = "2.4.16"
+  val akkaHttpVersion = "10.0.1"
+  val hadoopVersion = "2.6.0"
+  val hbaseVersion = "1.0.0"
+  val commonsHttpVersion = "3.1"
+  val commonsLoggingVersion = "1.1.3"
+  val commonsLangVersion = "2.6"
+  val commonsIOVersion = "2.4"
+  val dataReplicationVersion = "0.7"
+  val upickleVersion = "0.3.4"
+  val junitVersion = "4.12"
+  val kafkaVersion = "0.8.2.1"
+  val jsonSimpleVersion = "1.1"
+  val storm09Version = "0.9.6"
+  val stormVersion = "0.10.0"
+  val slf4jVersion = "1.7.16"
+  val guavaVersion = "16.0.1"
+  val codahaleVersion = "3.0.2"
+  val kryoVersion = "0.4.1"
+  val gsCollectionsVersion = "6.2.0"
+  val sprayVersion = "1.3.2"
+  val sprayJsonVersion = "1.3.1"
+  val scalaTestVersion = "2.2.0"
+  val scalaCheckVersion = "1.11.3"
+  val mockitoVersion = "1.10.17"
+  val bijectionVersion = "0.8.0"
+  val scalazVersion = "7.1.1"
+  val algebirdVersion = "0.9.0"
+  val chillVersion = "0.6.0"
+  val jedisVersion = "2.9.0"
+
+  val coreDependencies = Seq(
+    libraryDependencies ++= Seq(
+      "org.slf4j" % "slf4j-api" % slf4jVersion,
+      "org.slf4j" % "slf4j-log4j12" % slf4jVersion,
+      "commons-lang" % "commons-lang" % commonsLangVersion,
+
+      /**
+       * Overrides Netty version 3.10.3.Final used by Akka 2.4.2 to work-around netty hang issue
+       * (https://github.com/gearpump/gearpump/issues/2020)
+       *
+       * Akka 2.4.2 by default use Netty 3.10.3.Final, which has a serious issue which can hang
+       * the network. The same issue also happens in version range (3.10.0.Final, 3.10.5.Final)
+       * Netty 3.10.6.Final have this issue fixed, however, we find there is a 20% performance
+       * drop. So we decided to downgrade netty to 3.8.0.Final (Same version used in akka 2.3.12).
+       *
+       * @see https://github.com/gearpump/gearpump/pull/2017 for more discussions.
+       */
+      "io.netty" % "netty" % "3.8.0.Final",
+      "com.typesafe.akka" %% "akka-remote" % akkaVersion
+        exclude("io.netty", "netty"),
+
+      "com.typesafe.akka" %% "akka-cluster" % akkaVersion,
+      "com.typesafe.akka" %% "akka-cluster-tools" % akkaVersion,
+      "commons-logging" % "commons-logging" % commonsLoggingVersion,
+      "com.typesafe.akka" %% "akka-distributed-data-experimental" % akkaVersion,
+      "com.typesafe.akka" %% "akka-actor" % akkaVersion,
+      "com.typesafe.akka" %% "akka-agent" % akkaVersion,
+      "com.typesafe.akka" %% "akka-slf4j" % akkaVersion,
+      "com.typesafe.akka" %% "akka-kernel" % akkaVersion,
+      "com.typesafe.akka" %% "akka-http" % akkaHttpVersion,
+      "com.typesafe.akka" %% "akka-http-spray-json" % akkaHttpVersion,
+      "org.scala-lang" % "scala-reflect" % scalaVersionNumber,
+      "com.github.romix.akka" %% "akka-kryo-serialization" % kryoVersion,
+      "com.google.guava" % "guava" % guavaVersion,
+      "com.codahale.metrics" % "metrics-graphite" % codahaleVersion
+        exclude("org.slf4j", "slf4j-api"),
+      "com.codahale.metrics" % "metrics-jvm" % codahaleVersion
+        exclude("org.slf4j", "slf4j-api"),
+      "com.typesafe.akka" %% "akka-testkit" % akkaVersion % "test",
+      "org.scalatest" %% "scalatest" % scalaTestVersion % "test",
+      "org.scalacheck" %% "scalacheck" % scalaCheckVersion % "test",
+      "org.mockito" % "mockito-core" % mockitoVersion % "test",
+      "junit" % "junit" % junitVersion % "test"
+    )
+  )
+}
\ No newline at end of file
diff --git a/project/Docs.scala b/project/Docs.scala
new file mode 100644
index 0000000..f8d433e
--- /dev/null
+++ b/project/Docs.scala
@@ -0,0 +1,61 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import BuildGearpump.{core, streaming}
+import BuildExternals.externals
+import sbt.Keys._
+import sbt._
+import sbtunidoc.Plugin.UnidocKeys._
+import sbtunidoc.Plugin._
+
+object Docs extends sbt.Build {
+  lazy val javadocSettings = Seq(
+    addCompilerPlugin(
+      "com.typesafe.genjavadoc" %% "genjavadoc-plugin" % "0.9" cross CrossVersion.full),
+    scalacOptions += s"-P:genjavadoc:out=${target.value}/java"
+  )
+
+  lazy val gearpumpUnidocSetting = scalaJavaUnidocSettings ++ Seq(
+    unidocProjectFilter in(ScalaUnidoc, unidoc) := projectsWithDoc,
+    unidocProjectFilter in(JavaUnidoc, unidoc) := projectsWithDoc,
+
+    unidocAllSources in(ScalaUnidoc, unidoc) := {
+      ignoreUndocumentedPackages((unidocAllSources in(ScalaUnidoc, unidoc)).value)
+    },
+
+    // Skip class names containing $ and some internal packages in Javadocs
+    unidocAllSources in(JavaUnidoc, unidoc) := {
+      ignoreUndocumentedPackages((unidocAllSources in(JavaUnidoc, unidoc)).value)
+    }
+  )
+
+  private lazy val projectsWithDoc = {
+    val projects: Seq[ProjectReference] = Seq[ProjectReference](
+      core,
+      streaming
+    ) ++ externals
+
+    inProjects(projects: _*)
+  }
+
+  private def ignoreUndocumentedPackages(packages: Seq[Seq[File]]): Seq[Seq[File]] = {
+    packages
+      .map(_.filterNot(_.getName.contains("$")))
+      .map(_.filterNot(_.getCanonicalPath.contains("akka")))
+  }
+}
diff --git a/project/Pack.scala b/project/Pack.scala
index 1c87653..2eb0ca7 100644
--- a/project/Pack.scala
+++ b/project/Pack.scala
@@ -18,13 +18,14 @@
 
 import sbt.Keys._
 import sbt._
-import Build._
+import BuildGearpump._
+import BuildDashboard.services
+import BuildExperiments.{cgroup, storm, yarn}
 import xerial.sbt.Pack._
 
 object Pack extends sbt.Build {
   val daemonClassPath = Seq(
     "${PROG_HOME}/conf",
-    "${PROG_HOME}/lib/daemon/*",
     // This is for DFSJarStore
     "${PROG_HOME}/lib/yarn/*"
   )
@@ -37,14 +38,12 @@
 
   val serviceClassPath = Seq(
     "${PROG_HOME}/conf",
-    "${PROG_HOME}/lib/daemon/*",
     "${PROG_HOME}/lib/services/*",
     "${PROG_HOME}/dashboard"
   )
 
   val yarnClassPath = Seq(
     "${PROG_HOME}/conf",
-    "${PROG_HOME}/lib/daemon/*",
     "${PROG_HOME}/lib/services/*",
     "${PROG_HOME}/lib/yarn/*",
     "${PROG_HOME}/conf/yarnconf",
@@ -72,9 +71,14 @@
           "storm" -> "org.apache.gearpump.experiments.storm.StormRunner"
         ),
         packJvmOpts := Map(
-          "gear" -> Seq("-Djava.net.preferIPv4Stack=true", "-Dgearpump.home=${PROG_HOME}"),
+          "gear" -> Seq(
+            "-noverify",
+            "-Djava.net.preferIPv4Stack=true",
+            "-Dgearpump.home=${PROG_HOME}"),
+
           "local" -> Seq(
             "-server",
+            "-noverify",
             "-Djava.net.preferIPv4Stack=true",
             "-DlogFilename=local",
             "-Dgearpump.home=${PROG_HOME}",
@@ -82,6 +86,7 @@
 
           "master" -> Seq(
             "-server",
+            "-noverify",
             "-Djava.net.preferIPv4Stack=true",
             "-DlogFilename=master",
             "-Dgearpump.home=${PROG_HOME}",
@@ -89,6 +94,7 @@
 
           "worker" -> Seq(
             "-server",
+            "-noverify",
             "-Djava.net.preferIPv4Stack=true",
             "-DlogFilename=worker",
             "-Dgearpump.home=${PROG_HOME}",
@@ -96,26 +102,28 @@
 
           "services" -> Seq(
             "-server",
+            "-noverify",
             "-Djava.net.preferIPv4Stack=true",
             "-Dgearpump.home=${PROG_HOME}",
             "-Djava.rmi.server.hostname=localhost"),
 
           "yarnclient" -> Seq(
             "-server",
+            "-noverify",
             "-Djava.net.preferIPv4Stack=true",
             "-Dgearpump.home=${PROG_HOME}",
             "-Djava.rmi.server.hostname=localhost"),
 
           "storm" -> Seq(
             "-server",
+            "-noverify",
             "-Djava.net.preferIPv4Stack=true",
             "-Dgearpump.home=${PROG_HOME}")
         ),
         packLibDir := Map(
-          "lib" -> new ProjectsToPack(core.id, streaming.id),
-          "lib/daemon" -> new ProjectsToPack(daemon.id, cgroup.id).exclude(core.id, streaming.id),
-          "lib/yarn" -> new ProjectsToPack(yarn.id).exclude(services.id, daemon.id),
-          "lib/services" -> new ProjectsToPack(services.id).exclude(daemon.id),
+          "lib/yarn" -> new ProjectsToPack(gearpumpHadoop.id, yarn.id).
+            exclude(services.id, core.id),
+          "lib/services" -> new ProjectsToPack(services.id).exclude(core.id),
           "lib/storm" -> new ProjectsToPack(storm.id).exclude(streaming.id)
         ),
         packExclude := Seq(thisProjectRef.value.project),
@@ -123,7 +131,9 @@
         packResourceDir += (baseDirectory.value / ".." / "bin" -> "bin"),
         packResourceDir += (baseDirectory.value / ".." / "conf" -> "conf"),
         packResourceDir += (baseDirectory.value / ".." / "yarnconf" -> "conf/yarnconf"),
-        packResourceDir += (baseDirectory.value / ".." / "shaded" / "target" /
+        packResourceDir += (baseDirectory.value / ".." / "core" / "target" /
+          CrossVersion.binaryScalaVersion(scalaVersion.value) -> "lib"),
+        packResourceDir += (baseDirectory.value / ".." / "streaming" / "target" /
           CrossVersion.binaryScalaVersion(scalaVersion.value) -> "lib"),
         packResourceDir += (baseDirectory.value / ".." / "services" / "dashboard" -> "dashboard"),
         packResourceDir += (baseDirectory.value / ".." / "examples" / "target" /
@@ -138,7 +148,7 @@
           "gear" -> applicationClassPath,
           "local" -> daemonClassPath,
           "master" -> daemonClassPath,
-          "worker" -> daemonClassPath,
+          "worker" -> applicationClassPath,
           "services" -> serviceClassPath,
           "yarnclient" -> yarnClassPath,
           "storm" -> stormClassPath
@@ -148,6 +158,6 @@
         packArchiveExcludes := Seq("integrationtest")
 
       )
-  ).dependsOn(core, streaming, services, yarn, storm).
+  ).dependsOn(core, streaming, services, yarn, storm, cgroup).
     disablePlugins(sbtassembly.AssemblyPlugin)
 }
diff --git a/project/build.properties b/project/build.properties
index c4df008..8df00d1 100644
--- a/project/build.properties
+++ b/project/build.properties
@@ -16,4 +16,4 @@
 # limitations under the License.
 #
 
-sbt.version=0.13.11
+sbt.version=0.13.13
diff --git a/project/license.sbt b/project/license.sbt
deleted file mode 100644
index 3452eed..0000000
--- a/project/license.sbt
+++ /dev/null
@@ -1 +0,0 @@
-addSbtPlugin("com.typesafe.sbt" % "sbt-license-report" % "1.2.0")
diff --git a/project/plugins.sbt b/project/plugins.sbt
index 5a9740e..0a6a562 100644
--- a/project/plugins.sbt
+++ b/project/plugins.sbt
@@ -16,8 +16,7 @@
  * limitations under the License.
  */
 
-resolvers += Resolver.url("fvunicorn",
-  url("http://dl.bintray.com/fvunicorn/sbt-plugins"))(Resolver.ivyStylePatterns)
+resolvers += Resolver.bintrayIvyRepo("fvunicorn", "sbt-plugins")
 
 resolvers += Classpaths.sbtPluginReleases
 
@@ -25,7 +24,7 @@
 
 addSbtPlugin("com.eed3si9n" % "sbt-assembly" % "0.14.3")
 
-addSbtPlugin("io.gearpump.sbt" % "sbt-pack" % "0.7.6")
+addSbtPlugin("io.gearpump.sbt" % "sbt-pack" % "0.7.7")
 
 addSbtPlugin("de.johoop" % "jacoco4sbt" % "2.1.6")
 
diff --git a/scalastyle-config.xml b/scalastyle-config.xml
new file mode 100644
index 0000000..1b0a838
--- /dev/null
+++ b/scalastyle-config.xml
@@ -0,0 +1,240 @@
+<!--
+  ~ Licensed to the Apache Software Foundation (ASF) under one
+  ~ or more contributor license agreements.  See the NOTICE file
+  ~ distributed with this work for additional information
+  ~ regarding copyright ownership.  The ASF licenses this file
+  ~ to you under the Apache License, Version 2.0 (the
+  ~ "License"); you may not use this file except in compliance
+  ~ with the License.  You may obtain a copy of the License at
+  ~
+  ~      http://www.apache.org/licenses/LICENSE-2.0
+  ~
+  ~ Unless required by applicable law or agreed to in writing, software
+  ~ distributed under the License is distributed on an "AS IS" BASIS,
+  ~ WITHOUT 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:
+
+  // scalastyle:off
+  ...  // stuff that breaks the styles
+  // scalastyle:on
+
+You can also disable only one rule, by specifying its rule id, as specified in:
+  http://www.scalastyle.org/rules-0.8.0.html
+
+  // scalastyle:off no.finalize
+  override def finalize(): Unit = ...
+  // scalastyle:on no.finalize
+
+This file is divided into 3 sections:
+ (1) rules that we enforce.
+ (2) rules that we would like to enforce, but haven't cleaned up the codebase to turn on yet
+     (or we need to make the scalastyle rule more configurable).
+ (3) rules that we don't want to enforce.
+-->
+
+<scalastyle>
+  <name>Scalastyle standard configuration</name>
+
+  <check level="error" class="org.scalastyle.file.FileTabChecker" enabled="true"></check>
+
+  <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
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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>
+
+  <check level="error" class="org.scalastyle.scalariform.SpacesAfterPlusChecker"
+         enabled="true"></check>
+
+  <check level="error" class="org.scalastyle.scalariform.SpacesBeforePlusChecker"
+         enabled="true"></check>
+
+  <check level="error" class="org.scalastyle.file.FileLineLengthChecker" enabled="true">
+    <parameters>
+      <parameter name="maxLineLength"><![CDATA[100]]></parameter>
+      <parameter name="tabSize"><![CDATA[2]]></parameter>
+      <parameter name="ignoreImports">true</parameter>
+    </parameters>
+  </check>
+
+  <check level="error" class="org.scalastyle.scalariform.ClassNamesChecker" enabled="true">
+    <parameters>
+      <parameter name="regex"><![CDATA[[A-Z][A-Za-z]*]]></parameter>
+    </parameters>
+  </check>
+
+  <check level="error" class="org.scalastyle.scalariform.ObjectNamesChecker" enabled="true">
+    <parameters>
+      <parameter name="regex"><![CDATA[[A-Z][A-Za-z]*]]></parameter>
+    </parameters>
+  </check>
+
+  <check level="error" class="org.scalastyle.scalariform.PackageObjectNamesChecker" enabled="true">
+    <parameters>
+      <parameter name="regex"><![CDATA[^[a-z][A-Za-z]*$]]></parameter>
+    </parameters>
+  </check>
+
+  <check level="error" class="org.scalastyle.scalariform.ParameterNumberChecker" enabled="true">
+    <parameters>
+      <parameter name="maxParameters"><![CDATA[10]]></parameter>
+    </parameters>
+  </check>
+
+  <check level="error" class="org.scalastyle.scalariform.NoFinalizeChecker" enabled="true"></check>
+
+  <check level="error" class="org.scalastyle.scalariform.CovariantEqualsChecker"
+         enabled="true"></check>
+
+  <check level="error" class="org.scalastyle.scalariform.StructuralTypeChecker"
+         enabled="true"></check>
+
+  <check level="error" class="org.scalastyle.scalariform.UppercaseLChecker" enabled="true"></check>
+
+  <check level="error" class="org.scalastyle.scalariform.IfBraceChecker" enabled="true">
+    <parameters>
+      <parameter name="singleLineAllowed"><![CDATA[true]]></parameter>
+      <parameter name="doubleLineAllowed"><![CDATA[true]]></parameter>
+    </parameters>
+  </check>
+
+  <check level="error" class="org.scalastyle.scalariform.PublicMethodsHaveTypeChecker"
+         enabled="true"></check>
+
+  <check level="error" class="org.scalastyle.scalariform.NonASCIICharacterChecker"
+         enabled="true"></check>
+
+  <check level="error" class="org.scalastyle.scalariform.SpaceAfterCommentStartChecker"
+         enabled="true"></check>
+
+  <check level="error" class="org.scalastyle.scalariform.EnsureSingleSpaceBeforeTokenChecker"
+         enabled="true">
+    <parameters>
+      <parameter name="tokens">ARROW, EQUALS, ELSE, TRY, CATCH, FINALLY, LARROW, RARROW</parameter>
+    </parameters>
+  </check>
+
+  <check level="error" class="org.scalastyle.scalariform.EnsureSingleSpaceAfterTokenChecker"
+         enabled="true">
+    <parameters>
+      <parameter name="tokens">ARROW, EQUALS, COMMA, COLON, IF, ELSE, DO, WHILE, FOR, MATCH, TRY,
+        CATCH, FINALLY, LARROW, RARROW
+      </parameter>
+    </parameters>
+  </check>
+
+  <!-- ??? usually shouldn't be checked into the code base. -->
+  <check level="error" class="org.scalastyle.scalariform.NotImplementedErrorUsage"
+         enabled="true"></check>
+
+  <check customId="println" level="error" class="org.scalastyle.scalariform.TokenChecker"
+         enabled="true">
+    <parameters>
+      <parameter name="regex">^println$</parameter>
+    </parameters>
+    <customMessage><![CDATA[Are you sure you want to println? If yes, wrap the code block with
+      // scalastyle:off println
+      println(...)
+      // scalastyle:on println]]></customMessage>
+  </check>
+
+  <check customId="runtimeaddshutdownhook" level="error" class="org.scalastyle.file.RegexChecker"
+         enabled="true">
+    <parameters>
+      <parameter name="regex">Runtime\.getRuntime\.addShutdownHook</parameter>
+    </parameters>
+    <customMessage><![CDATA[
+      Are you sure that you want to use Runtime.getRuntime.addShutdownHook? In most cases, you should use
+      ShutdownHookManager.addShutdownHook instead.
+      If you must use Runtime.getRuntime.addShutdownHook, wrap the code block with
+      // scalastyle:off runtimeaddshutdownhook
+      Runtime.getRuntime.addShutdownHook(...)
+      // scalastyle:on runtimeaddshutdownhook
+    ]]></customMessage>
+  </check>
+
+  <check customId="mutablesynchronizedbuffer" level="error" class="org.scalastyle.file.RegexChecker"
+         enabled="true">
+    <parameters>
+      <parameter name="regex">mutable\.SynchronizedBuffer</parameter>
+    </parameters>
+    <customMessage><![CDATA[
+      Are you sure that you want to use mutable.SynchronizedBuffer? In most cases, you should use
+      java.util.concurrent.ConcurrentLinkedQueue instead.
+      If you must use mutable.SynchronizedBuffer, wrap the code block with
+      // scalastyle:off mutablesynchronizedbuffer
+      mutable.SynchronizedBuffer[...]
+      // scalastyle:on mutablesynchronizedbuffer
+    ]]></customMessage>
+  </check>
+
+  <check customId="javaconversions" level="error" class="org.scalastyle.scalariform.TokenChecker"
+         enabled="true">
+    <parameters>
+      <parameter name="regex">JavaConversions</parameter>
+    </parameters>
+    <customMessage>Instead of importing implicits in scala.collection.JavaConversions._, import
+      scala.collection.JavaConverters._ and use .asScala / .asJava methods
+    </customMessage>
+  </check>
+
+  <check level="error" class="org.scalastyle.scalariform.DisallowSpaceBeforeTokenChecker"
+         enabled="true">
+    <parameters>
+      <parameter name="tokens">COMMA</parameter>
+    </parameters>
+  </check>
+
+  <!-- Should add single Space between ')' and '{' -->
+  <check customId="SingleSpaceBetweenRParenAndLCurlyBrace" level="error"
+         class="org.scalastyle.file.RegexChecker" enabled="true">
+    <parameters>
+      <parameter name="regex">\)\{</parameter>
+    </parameters>
+    <customMessage><![CDATA[
+      Single Space between ')' and `{`.
+    ]]></customMessage>
+  </check>
+
+  <check level="error" class="org.scalastyle.scalariform.MethodNamesChecker" enabled="false">
+    <parameters>
+      <parameter name="regex"><![CDATA[^[a-z][A-Za-z0-9]*$]]></parameter>
+    </parameters>
+  </check>
+
+  <check level="error" class="org.scalastyle.scalariform.EqualsHashCodeChecker"
+         enabled="true"></check>
+
+  <check level="error" class="org.scalastyle.file.IndentationChecker" enabled="true">
+    <parameters>
+      <parameter name="tabSize">2</parameter>
+      <parameter name="methodParamIndentSize">4</parameter>
+    </parameters>
+  </check>
+
+  <!-- Don't allow return -->
+  <check level="error" class="org.scalastyle.scalariform.ReturnChecker" enabled="true"></check>
+
+</scalastyle>
\ No newline at end of file
diff --git a/services/jvm/src/main/scala/org/apache/gearpump/services/AppMasterService.scala b/services/jvm/src/main/scala/org/apache/gearpump/services/AppMasterService.scala
index 46e16cf..53ee692 100644
--- a/services/jvm/src/main/scala/org/apache/gearpump/services/AppMasterService.scala
+++ b/services/jvm/src/main/scala/org/apache/gearpump/services/AppMasterService.scala
@@ -196,14 +196,6 @@
       } ~
       pathEnd {
         delete {
-          val writer = (result: ShutdownApplicationResult) => {
-            val output = if (result.appId.isSuccess) {
-              Map("status" -> "success", "info" -> null)
-            } else {
-              Map("status" -> "fail", "info" -> result.appId.failed.get.toString)
-            }
-            write(output)
-          }
           onComplete(askActor[ShutdownApplicationResult](master, ShutdownApplication(appId))) {
             case Success(result) =>
               val output = if (result.appId.isSuccess) {
@@ -223,4 +215,4 @@
 
 object AppMasterService {
   case class Status(success: Boolean, reason: String = null)
-}
\ No newline at end of file
+}
diff --git a/services/jvm/src/main/scala/org/apache/gearpump/services/MasterService.scala b/services/jvm/src/main/scala/org/apache/gearpump/services/MasterService.scala
index a763be6..be96577 100644
--- a/services/jvm/src/main/scala/org/apache/gearpump/services/MasterService.scala
+++ b/services/jvm/src/main/scala/org/apache/gearpump/services/MasterService.scala
@@ -41,7 +41,7 @@
 import org.apache.gearpump.cluster.worker.WorkerSummary
 import org.apache.gearpump.cluster.{ClusterConfig, UserConfig}
 import org.apache.gearpump.jarstore.{JarStoreClient, FileDirective, JarStoreServer}
-import org.apache.gearpump.partitioner.{PartitionerByClassName, PartitionerDescription}
+import org.apache.gearpump.streaming.partitioner.{PartitionerByClassName, PartitionerDescription}
 import org.apache.gearpump.services.MasterService.{BuiltinPartitioners, SubmitApplicationRequest}
 // NOTE: This cannot be removed!!!
 import org.apache.gearpump.services.util.UpickleUtil._
@@ -159,7 +159,7 @@
         entity(as[String]) { request =>
           val msg = java.net.URLDecoder.decode(request, "UTF-8")
           val submitApplicationRequest = read[SubmitApplicationRequest](msg)
-          import submitApplicationRequest.{appName, dag, processors, userconfig}
+          import submitApplicationRequest.{appName, dag, processors, userConfig}
           val context = ClientContext(system.settings.config, system, master)
 
           val graph = dag.mapVertex { processorId =>
@@ -168,8 +168,8 @@
             PartitionerDescription(new PartitionerByClassName(edge))
           }
 
-          val effectiveConfig = if (userconfig == null) UserConfig.empty else userconfig
-          val appId = context.submit(new StreamApplication(appName, effectiveConfig, graph))
+          val effectiveConfig = if (userConfig == null) UserConfig.empty else userConfig
+          val appId = context.submit(new StreamApplication(appName, effectiveConfig, graph)).appId
 
           import upickle.default.write
           val submitApplicationResultValue = SubmitApplicationResultValue(appId)
@@ -192,7 +192,8 @@
     } ~
     path("partitioners") {
       get {
-        complete(write(BuiltinPartitioners(Constants.BUILTIN_PARTITIONERS.map(_.getName))))
+        complete(write(BuiltinPartitioners(org.apache.gearpump.streaming.Constants
+          .BUILTIN_PARTITIONERS.map(_.getName))))
       }
     }
   }
@@ -344,5 +345,5 @@
       appName: String,
       processors: Map[ProcessorId, ProcessorDescription],
       dag: Graph[Int, String],
-      userconfig: UserConfig)
+      userConfig: UserConfig)
 }
diff --git a/services/jvm/src/main/scala/org/apache/gearpump/services/SecurityService.scala b/services/jvm/src/main/scala/org/apache/gearpump/services/SecurityService.scala
index 8ae8dbe..4989364 100644
--- a/services/jvm/src/main/scala/org/apache/gearpump/services/SecurityService.scala
+++ b/services/jvm/src/main/scala/org/apache/gearpump/services/SecurityService.scala
@@ -61,7 +61,7 @@
 
   // Use scheme "GearpumpBasic" to avoid popping up web browser native authentication box.
   private val challenge = HttpChallenge(scheme = "GearpumpBasic", realm = Some("gearpump"),
-    params = Map.empty[String,String])
+    params = Map.empty[String, String])
 
   val LOG = LogUtil.getLogger(getClass, "AUDIT")
 
diff --git a/streaming/src/main/java/org/apache/gearpump/streaming/javaapi/Graph.java b/streaming/src/main/java/org/apache/gearpump/streaming/javaapi/Graph.java
index 8f85aa3..aaf6db8 100644
--- a/streaming/src/main/java/org/apache/gearpump/streaming/javaapi/Graph.java
+++ b/streaming/src/main/java/org/apache/gearpump/streaming/javaapi/Graph.java
@@ -18,7 +18,7 @@
 
 package org.apache.gearpump.streaming.javaapi;
 
-import org.apache.gearpump.partitioner.Partitioner;
+import org.apache.gearpump.streaming.partitioner.Partitioner;
 import org.apache.gearpump.streaming.Processor;
 import org.apache.gearpump.streaming.task.Task;
 
diff --git a/streaming/src/main/java/org/apache/gearpump/streaming/javaapi/dsl/functions/FilterFunction.java b/streaming/src/main/java/org/apache/gearpump/streaming/javaapi/dsl/functions/FilterFunction.java
deleted file mode 100644
index f07ceff..0000000
--- a/streaming/src/main/java/org/apache/gearpump/streaming/javaapi/dsl/functions/FilterFunction.java
+++ /dev/null
@@ -1,30 +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 org.apache.gearpump.streaming.javaapi.dsl.functions;
-
-import java.io.Serializable;
-
-/**
- * Filter function
- *
- * @param <T> Message of type T
- */
-public interface FilterFunction<T> extends Serializable {
-  boolean apply(T t);
-}
diff --git a/streaming/src/main/java/org/apache/gearpump/streaming/javaapi/dsl/functions/FlatMapFunction.java b/streaming/src/main/java/org/apache/gearpump/streaming/javaapi/dsl/functions/FlatMapFunction.java
deleted file mode 100644
index 9788dd2..0000000
--- a/streaming/src/main/java/org/apache/gearpump/streaming/javaapi/dsl/functions/FlatMapFunction.java
+++ /dev/null
@@ -1,32 +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 org.apache.gearpump.streaming.javaapi.dsl.functions;
-
-import java.io.Serializable;
-import java.util.Iterator;
-
-/**
- * Function that converts a value of type T to a iterator of values of type R.
- *
- * @param <T> Input value type
- * @param <R> Return value type
- */
-public interface FlatMapFunction<T, R> extends Serializable {
-  Iterator<R> apply(T t);
-}
diff --git a/streaming/src/main/java/org/apache/gearpump/streaming/javaapi/dsl/functions/MapFunction.java b/streaming/src/main/java/org/apache/gearpump/streaming/javaapi/dsl/functions/MapFunction.java
deleted file mode 100644
index e1fc821..0000000
--- a/streaming/src/main/java/org/apache/gearpump/streaming/javaapi/dsl/functions/MapFunction.java
+++ /dev/null
@@ -1,31 +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 org.apache.gearpump.streaming.javaapi.dsl.functions;
-
-import java.io.Serializable;
-
-/**
- * Function that map a value of type T to value of type R
- *
- * @param <T> Input value type
- * @param <R> Output value type
- */
-public interface MapFunction<T, R> extends Serializable {
-  R apply(T t);
-}
diff --git a/streaming/src/main/java/org/apache/gearpump/streaming/javaapi/dsl/functions/ReduceFunction.java b/streaming/src/main/java/org/apache/gearpump/streaming/javaapi/dsl/functions/ReduceFunction.java
deleted file mode 100644
index 2bcac60..0000000
--- a/streaming/src/main/java/org/apache/gearpump/streaming/javaapi/dsl/functions/ReduceFunction.java
+++ /dev/null
@@ -1,30 +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 org.apache.gearpump.streaming.javaapi.dsl.functions;
-
-import java.io.Serializable;
-
-/**
- * Function that applies reduce operation
- *
- * @param <T> Input value type
- */
-public interface ReduceFunction<T> extends Serializable {
-  T apply(T t1, T t2);
-}
diff --git a/streaming/src/main/scala/org/apache/gearpump/streaming/Constants.scala b/streaming/src/main/scala/org/apache/gearpump/streaming/Constants.scala
index f99a436..d7582b0 100644
--- a/streaming/src/main/scala/org/apache/gearpump/streaming/Constants.scala
+++ b/streaming/src/main/scala/org/apache/gearpump/streaming/Constants.scala
@@ -17,6 +17,8 @@
  */
 package org.apache.gearpump.streaming
 
+import org.apache.gearpump.streaming.partitioner._
+
 object Constants {
 
   val GEARPUMP_STREAMING_OPERATOR = "gearpump.streaming.dsl.operator"
@@ -36,4 +38,12 @@
 
   val GEARPUMP_STREAMING_EXECUTOR_RESTART_TIME_WINDOW =
     "gearpump.streaming.executor-restart-time-window"
+
+  // The partitioners provided by Gearpump
+  val BUILTIN_PARTITIONERS = Array(
+    classOf[BroadcastPartitioner],
+    classOf[CoLocationPartitioner],
+    classOf[HashPartitioner],
+    classOf[ShuffleGroupingPartitioner],
+    classOf[ShufflePartitioner])
 }
diff --git a/streaming/src/main/scala/org/apache/gearpump/streaming/DAG.scala b/streaming/src/main/scala/org/apache/gearpump/streaming/DAG.scala
index 4a94ad3..8ad74f8 100644
--- a/streaming/src/main/scala/org/apache/gearpump/streaming/DAG.scala
+++ b/streaming/src/main/scala/org/apache/gearpump/streaming/DAG.scala
@@ -18,7 +18,7 @@
 
 package org.apache.gearpump.streaming
 
-import org.apache.gearpump.partitioner.PartitionerDescription
+import org.apache.gearpump.streaming.partitioner.PartitionerDescription
 import org.apache.gearpump.streaming.task.TaskId
 import org.apache.gearpump.util.Graph
 
diff --git a/streaming/src/main/scala/org/apache/gearpump/streaming/StreamApplication.scala b/streaming/src/main/scala/org/apache/gearpump/streaming/StreamApplication.scala
index 66ec873..d4b3719 100644
--- a/streaming/src/main/scala/org/apache/gearpump/streaming/StreamApplication.scala
+++ b/streaming/src/main/scala/org/apache/gearpump/streaming/StreamApplication.scala
@@ -25,7 +25,7 @@
 
 import org.apache.gearpump.TimeStamp
 import org.apache.gearpump.cluster._
-import org.apache.gearpump.partitioner.{HashPartitioner, Partitioner, PartitionerDescription, PartitionerObject}
+import org.apache.gearpump.streaming.partitioner.{HashPartitioner, Partitioner, PartitionerDescription, PartitionerObject}
 import org.apache.gearpump.streaming.appmaster.AppMaster
 import org.apache.gearpump.streaming.task.Task
 import org.apache.gearpump.util.{Graph, LogUtil, ReferenceEqual}
diff --git a/streaming/src/main/scala/org/apache/gearpump/streaming/appmaster/AppMaster.scala b/streaming/src/main/scala/org/apache/gearpump/streaming/appmaster/AppMaster.scala
index 31e1151..1341464 100644
--- a/streaming/src/main/scala/org/apache/gearpump/streaming/appmaster/AppMaster.scala
+++ b/streaming/src/main/scala/org/apache/gearpump/streaming/appmaster/AppMaster.scala
@@ -30,7 +30,7 @@
 import org.apache.gearpump.cluster.worker.WorkerId
 import org.apache.gearpump.metrics.Metrics.ReportMetrics
 import org.apache.gearpump.metrics.{JvmMetricsSet, Metrics, MetricsReporterService}
-import org.apache.gearpump.partitioner.PartitionerDescription
+import org.apache.gearpump.streaming.partitioner.PartitionerDescription
 import org.apache.gearpump.streaming.ExecutorToAppMaster.{MessageLoss, RegisterExecutor, RegisterTask, UnRegisterTask}
 import org.apache.gearpump.streaming._
 import org.apache.gearpump.streaming.appmaster.AppMaster._
diff --git a/streaming/src/main/scala/org/apache/gearpump/streaming/appmaster/ClockService.scala b/streaming/src/main/scala/org/apache/gearpump/streaming/appmaster/ClockService.scala
index 68db354..2085953 100644
--- a/streaming/src/main/scala/org/apache/gearpump/streaming/appmaster/ClockService.scala
+++ b/streaming/src/main/scala/org/apache/gearpump/streaming/appmaster/ClockService.scala
@@ -23,7 +23,7 @@
 import java.util.concurrent.TimeUnit
 
 import akka.actor.{Actor, Cancellable, Stash}
-import org.apache.gearpump.google.common.primitives.Longs
+import com.google.common.primitives.Longs
 import org.apache.gearpump.TimeStamp
 import org.apache.gearpump.cluster.ClientToMaster.GetStallingTasks
 import org.apache.gearpump.streaming.AppMasterToMaster.StallingTasks
diff --git a/streaming/src/main/scala/org/apache/gearpump/streaming/appmaster/DagManager.scala b/streaming/src/main/scala/org/apache/gearpump/streaming/appmaster/DagManager.scala
index 2736f5e..6154946 100644
--- a/streaming/src/main/scala/org/apache/gearpump/streaming/appmaster/DagManager.scala
+++ b/streaming/src/main/scala/org/apache/gearpump/streaming/appmaster/DagManager.scala
@@ -21,7 +21,7 @@
 import akka.actor.{ExtendedActorSystem, Actor, ActorRef, Stash}
 import akka.serialization.JavaSerializer
 import org.apache.gearpump.cluster.UserConfig
-import org.apache.gearpump.partitioner.PartitionerDescription
+import org.apache.gearpump.streaming.partitioner.PartitionerDescription
 import org.apache.gearpump.streaming._
 import org.apache.gearpump.streaming.appmaster.DagManager._
 import org.apache.gearpump.streaming.storage.AppDataStore
diff --git a/streaming/src/main/scala/org/apache/gearpump/streaming/appmaster/JarScheduler.scala b/streaming/src/main/scala/org/apache/gearpump/streaming/appmaster/JarScheduler.scala
index 6de5306..e023cdf 100644
--- a/streaming/src/main/scala/org/apache/gearpump/streaming/appmaster/JarScheduler.scala
+++ b/streaming/src/main/scala/org/apache/gearpump/streaming/appmaster/JarScheduler.scala
@@ -24,7 +24,7 @@
 import org.apache.gearpump.cluster.AppJar
 import org.apache.gearpump.cluster.scheduler.{Resource, ResourceRequest}
 import org.apache.gearpump.cluster.worker.WorkerId
-import org.apache.gearpump.partitioner.PartitionerDescription
+import org.apache.gearpump.streaming.partitioner.PartitionerDescription
 import org.apache.gearpump.streaming.appmaster.JarScheduler._
 import org.apache.gearpump.streaming.task.TaskId
 import org.apache.gearpump.streaming.{DAG, ProcessorDescription}
diff --git a/core/src/main/scala/org/apache/gearpump/partitioner/CoLocationPartitioner.scala b/streaming/src/main/scala/org/apache/gearpump/streaming/dsl/api/functions/FilterFunction.scala
similarity index 61%
copy from core/src/main/scala/org/apache/gearpump/partitioner/CoLocationPartitioner.scala
copy to streaming/src/main/scala/org/apache/gearpump/streaming/dsl/api/functions/FilterFunction.scala
index 5a3eec4..e4e7309 100644
--- a/core/src/main/scala/org/apache/gearpump/partitioner/CoLocationPartitioner.scala
+++ b/streaming/src/main/scala/org/apache/gearpump/streaming/dsl/api/functions/FilterFunction.scala
@@ -15,17 +15,28 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+package org.apache.gearpump.streaming.dsl.api.functions
 
-package org.apache.gearpump.partitioner
+import org.apache.gearpump.streaming.dsl.scalaapi.functions.SerializableFunction
 
-import org.apache.gearpump.Message
+object FilterFunction {
+
+  def apply[T](fn: T => Boolean): FilterFunction[T] = {
+    new FilterFunction[T] {
+      override def apply(t: T): Boolean = {
+        fn(t)
+      }
+    }
+  }
+}
 
 /**
- * Will have the same parallelism with last processor
- * And each task in current processor will co-locate with task of last processor
+ * Returns true to keep the input and false otherwise.
+ *
+ * @param T Input value type
  */
-class CoLocationPartitioner extends UnicastPartitioner {
-  override def getPartition(msg: Message, partitionNum: Int, currentPartitionId: Int): Int = {
-    currentPartitionId
-  }
+abstract class FilterFunction[T] extends SerializableFunction {
+
+  def apply(t: T): Boolean
+
 }
diff --git a/core/src/main/scala/org/apache/gearpump/partitioner/CoLocationPartitioner.scala b/streaming/src/main/scala/org/apache/gearpump/streaming/dsl/api/functions/MapFunction.scala
similarity index 60%
copy from core/src/main/scala/org/apache/gearpump/partitioner/CoLocationPartitioner.scala
copy to streaming/src/main/scala/org/apache/gearpump/streaming/dsl/api/functions/MapFunction.scala
index 5a3eec4..70fe9d4 100644
--- a/core/src/main/scala/org/apache/gearpump/partitioner/CoLocationPartitioner.scala
+++ b/streaming/src/main/scala/org/apache/gearpump/streaming/dsl/api/functions/MapFunction.scala
@@ -15,17 +15,29 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+package org.apache.gearpump.streaming.dsl.api.functions
 
-package org.apache.gearpump.partitioner
+import org.apache.gearpump.streaming.dsl.scalaapi.functions.SerializableFunction
 
-import org.apache.gearpump.Message
+object MapFunction {
+
+  def apply[T, R](fn: T => R): MapFunction[T, R] = {
+    new MapFunction[T, R] {
+      override def apply(t: T): R = {
+        fn(t)
+      }
+    }
+  }
+}
 
 /**
- * Will have the same parallelism with last processor
- * And each task in current processor will co-locate with task of last processor
+ * Transforms an input into an output of possibly different types.
+ *
+ * @param T Input value type
+ * @param R Output value type
  */
-class CoLocationPartitioner extends UnicastPartitioner {
-  override def getPartition(msg: Message, partitionNum: Int, currentPartitionId: Int): Int = {
-    currentPartitionId
-  }
+abstract class MapFunction[T, R] extends SerializableFunction {
+
+  def apply(t: T): R
+
 }
diff --git a/core/src/main/scala/org/apache/gearpump/partitioner/CoLocationPartitioner.scala b/streaming/src/main/scala/org/apache/gearpump/streaming/dsl/api/functions/ReduceFunction.scala
similarity index 60%
copy from core/src/main/scala/org/apache/gearpump/partitioner/CoLocationPartitioner.scala
copy to streaming/src/main/scala/org/apache/gearpump/streaming/dsl/api/functions/ReduceFunction.scala
index 5a3eec4..25b12be 100644
--- a/core/src/main/scala/org/apache/gearpump/partitioner/CoLocationPartitioner.scala
+++ b/streaming/src/main/scala/org/apache/gearpump/streaming/dsl/api/functions/ReduceFunction.scala
@@ -15,17 +15,28 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+package org.apache.gearpump.streaming.dsl.api.functions
 
-package org.apache.gearpump.partitioner
+import org.apache.gearpump.streaming.dsl.scalaapi.functions.SerializableFunction
 
-import org.apache.gearpump.Message
+object ReduceFunction {
+
+  def apply[T](fn: (T, T) => T): ReduceFunction[T] = {
+    new ReduceFunction[T] {
+      override def apply(t1: T, t2: T): T = {
+        fn(t1, t2)
+      }
+    }
+  }
+}
 
 /**
- * Will have the same parallelism with last processor
- * And each task in current processor will co-locate with task of last processor
+ * Combines two inputs into one output of the same type.
+ *
+ * @param T Type of both inputs and output
  */
-class CoLocationPartitioner extends UnicastPartitioner {
-  override def getPartition(msg: Message, partitionNum: Int, currentPartitionId: Int): Int = {
-    currentPartitionId
-  }
+abstract class ReduceFunction[T] extends SerializableFunction {
+
+  def apply(t1: T, t2: T): T
+
 }
diff --git a/streaming/src/main/scala/org/apache/gearpump/streaming/dsl/javaapi/JavaStream.scala b/streaming/src/main/scala/org/apache/gearpump/streaming/dsl/javaapi/JavaStream.scala
index 3003b98..7f3c250 100644
--- a/streaming/src/main/scala/org/apache/gearpump/streaming/dsl/javaapi/JavaStream.scala
+++ b/streaming/src/main/scala/org/apache/gearpump/streaming/dsl/javaapi/JavaStream.scala
@@ -15,14 +15,14 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-
 package org.apache.gearpump.streaming.dsl.javaapi
 
-import scala.collection.JavaConverters._
 import org.apache.gearpump.cluster.UserConfig
+import org.apache.gearpump.streaming.dsl.api.functions.{FilterFunction, MapFunction, ReduceFunction}
+import org.apache.gearpump.streaming.dsl.javaapi.functions.{FlatMapFunction => JFlatMapFunction, GroupByFunction}
+import org.apache.gearpump.streaming.dsl.scalaapi.functions.FlatMapFunction
+import org.apache.gearpump.streaming.dsl.scalaapi.{Stream, WindowStream}
 import org.apache.gearpump.streaming.dsl.window.api.Window
-import org.apache.gearpump.streaming.dsl.{Stream, WindowStream}
-import org.apache.gearpump.streaming.javaapi.dsl.functions._
 import org.apache.gearpump.streaming.task.Task
 
 /**
@@ -31,23 +31,23 @@
 class JavaStream[T](val stream: Stream[T]) {
 
   /** FlatMap on stream */
-  def flatMap[R](fn: FlatMapFunction[T, R], description: String): JavaStream[R] = {
-    new JavaStream[R](stream.flatMap({ t: T => fn(t).asScala }, description))
+  def flatMap[R](fn: JFlatMapFunction[T, R], description: String): JavaStream[R] = {
+    new JavaStream[R](stream.flatMap(FlatMapFunction(fn), "flatMap"))
   }
 
   /** Map on stream */
   def map[R](fn: MapFunction[T, R], description: String): JavaStream[R] = {
-    new JavaStream[R](stream.map({ t: T => fn(t) }, description))
+    new JavaStream[R](stream.flatMap(FlatMapFunction(fn), description))
   }
 
   /** Only keep the messages that FilterFunction returns true.  */
   def filter(fn: FilterFunction[T], description: String): JavaStream[T] = {
-    new JavaStream[T](stream.filter({ t: T => fn(t) }, description))
+    new JavaStream[T](stream.flatMap(FlatMapFunction(fn), description))
   }
 
   /** Does aggregation on the stream */
   def reduce(fn: ReduceFunction[T], description: String): JavaStream[T] = {
-    new JavaStream[T](stream.reduce({ (t1: T, t2: T) => fn(t1, t2) }, description))
+    new JavaStream[T](stream.reduce(fn, description))
   }
 
   def log(): Unit = {
@@ -65,7 +65,7 @@
    */
   def groupBy[GROUP](fn: GroupByFunction[T, GROUP],
       parallelism: Int, description: String): JavaStream[T] = {
-    new JavaStream[T](stream.groupBy((t: T) => fn, parallelism, description))
+    new JavaStream[T](stream.groupBy(fn.apply, parallelism, description))
   }
 
   def window(win: Window, description: String): JavaWindowStream[T] = {
@@ -84,6 +84,6 @@
 
   def groupBy[GROUP](fn: GroupByFunction[T, GROUP], parallelism: Int,
       description: String): JavaStream[T] = {
-    new JavaStream[T](stream.groupBy((t: T) => fn, parallelism, description))
+    new JavaStream[T](stream.groupBy(fn.apply, parallelism, description))
   }
 }
diff --git a/streaming/src/main/scala/org/apache/gearpump/streaming/dsl/javaapi/JavaStreamApp.scala b/streaming/src/main/scala/org/apache/gearpump/streaming/dsl/javaapi/JavaStreamApp.scala
index 0d841be..f5b2910 100644
--- a/streaming/src/main/scala/org/apache/gearpump/streaming/dsl/javaapi/JavaStreamApp.scala
+++ b/streaming/src/main/scala/org/apache/gearpump/streaming/dsl/javaapi/JavaStreamApp.scala
@@ -19,13 +19,14 @@
 package org.apache.gearpump.streaming.dsl.javaapi
 
 import java.util.Collection
-import scala.collection.JavaConverters._
 
 import org.apache.gearpump.cluster.UserConfig
-import org.apache.gearpump.cluster.client.ClientContext
-import org.apache.gearpump.streaming.dsl.{CollectionDataSource, StreamApp}
+import org.apache.gearpump.cluster.client.{ClientContext, RunningApplication}
+import org.apache.gearpump.streaming.dsl.scalaapi.{CollectionDataSource, StreamApp}
 import org.apache.gearpump.streaming.source.DataSource
 
+import scala.collection.JavaConverters._
+
 class JavaStreamApp(name: String, context: ClientContext, userConfig: UserConfig) {
 
   private val streamApp = StreamApp(name, context, userConfig)
@@ -41,7 +42,7 @@
     new JavaStream[T](streamApp.source(dataSource, parallelism, conf, description))
   }
 
-  def run(): Unit = {
+  def submit(): RunningApplication = {
     context.submit(streamApp)
   }
 }
diff --git a/core/src/main/scala/org/apache/gearpump/partitioner/CoLocationPartitioner.scala b/streaming/src/main/scala/org/apache/gearpump/streaming/dsl/javaapi/functions/FlatMapFunction.scala
similarity index 63%
copy from core/src/main/scala/org/apache/gearpump/partitioner/CoLocationPartitioner.scala
copy to streaming/src/main/scala/org/apache/gearpump/streaming/dsl/javaapi/functions/FlatMapFunction.scala
index 5a3eec4..85d597d 100644
--- a/core/src/main/scala/org/apache/gearpump/partitioner/CoLocationPartitioner.scala
+++ b/streaming/src/main/scala/org/apache/gearpump/streaming/dsl/javaapi/functions/FlatMapFunction.scala
@@ -15,17 +15,18 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+package org.apache.gearpump.streaming.dsl.javaapi.functions
 
-package org.apache.gearpump.partitioner
-
-import org.apache.gearpump.Message
+import org.apache.gearpump.streaming.dsl.scalaapi.functions.SerializableFunction
 
 /**
- * Will have the same parallelism with last processor
- * And each task in current processor will co-locate with task of last processor
+ * Transforms one input into zero or more outputs of possibly different types.
+ * This Java version of FlatMapFunction returns a java.util.Iterator.
+ *
+ * @param T Input value type
+ * @param R Output value type
  */
-class CoLocationPartitioner extends UnicastPartitioner {
-  override def getPartition(msg: Message, partitionNum: Int, currentPartitionId: Int): Int = {
-    currentPartitionId
-  }
+abstract class FlatMapFunction[T, R] extends SerializableFunction {
+
+  def apply(t: T): java.util.Iterator[R]
 }
diff --git a/streaming/src/main/java/org/apache/gearpump/streaming/javaapi/dsl/functions/GroupByFunction.java b/streaming/src/main/scala/org/apache/gearpump/streaming/dsl/javaapi/functions/GroupByFunction.scala
similarity index 72%
rename from streaming/src/main/java/org/apache/gearpump/streaming/javaapi/dsl/functions/GroupByFunction.java
rename to streaming/src/main/scala/org/apache/gearpump/streaming/dsl/javaapi/functions/GroupByFunction.scala
index 6c71280..7656cba 100644
--- a/streaming/src/main/java/org/apache/gearpump/streaming/javaapi/dsl/functions/GroupByFunction.java
+++ b/streaming/src/main/scala/org/apache/gearpump/streaming/dsl/javaapi/functions/GroupByFunction.scala
@@ -15,17 +15,14 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+package org.apache.gearpump.streaming.dsl.javaapi.functions
 
-package org.apache.gearpump.streaming.javaapi.dsl.functions;
-
-import java.io.Serializable;
+import org.apache.gearpump.streaming.dsl.api.functions.MapFunction
 
 /**
- * GroupBy function which assign value of type T to groups
+ * Assigns the input value into a group.
  *
- * @param <T> Input value type
- * @param <Group> Group Type
+ * @param T Input value type
+ * @param GROUP Group value type
  */
-public interface GroupByFunction<T, Group> extends Serializable {
-  Group apply(T t);
-}
+abstract class GroupByFunction[T, GROUP] extends MapFunction[T, GROUP]
diff --git a/streaming/src/main/scala/org/apache/gearpump/streaming/dsl/partitioner/GroupByPartitioner.scala b/streaming/src/main/scala/org/apache/gearpump/streaming/dsl/partitioner/GroupByPartitioner.scala
index 2ec881b..efa7409 100644
--- a/streaming/src/main/scala/org/apache/gearpump/streaming/dsl/partitioner/GroupByPartitioner.scala
+++ b/streaming/src/main/scala/org/apache/gearpump/streaming/dsl/partitioner/GroupByPartitioner.scala
@@ -19,7 +19,7 @@
 package org.apache.gearpump.streaming.dsl.partitioner
 
 import org.apache.gearpump.Message
-import org.apache.gearpump.partitioner.UnicastPartitioner
+import org.apache.gearpump.streaming.partitioner.UnicastPartitioner
 import org.apache.gearpump.streaming.dsl.window.api.GroupByFn
 
 /**
diff --git a/streaming/src/main/scala/org/apache/gearpump/streaming/dsl/plan/OP.scala b/streaming/src/main/scala/org/apache/gearpump/streaming/dsl/plan/OP.scala
index b2c5506..5aaf2fa 100644
--- a/streaming/src/main/scala/org/apache/gearpump/streaming/dsl/plan/OP.scala
+++ b/streaming/src/main/scala/org/apache/gearpump/streaming/dsl/plan/OP.scala
@@ -22,11 +22,10 @@
 import org.apache.gearpump.cluster.UserConfig
 import org.apache.gearpump.streaming.Constants._
 import org.apache.gearpump.streaming.Processor.DefaultProcessor
-import org.apache.gearpump.streaming.dsl.plan.functions.SingleInputFunction
+import org.apache.gearpump.streaming.dsl.plan.functions.{AndThen, SingleInputFunction}
 import org.apache.gearpump.streaming.{Constants, Processor}
 import org.apache.gearpump.streaming.dsl.task.TransformTask
-import org.apache.gearpump.streaming.dsl.window.api.{CountWindow, GroupByFn}
-import org.apache.gearpump.streaming.dsl.window.impl.GroupAlsoByWindow
+import org.apache.gearpump.streaming.dsl.window.api.GroupByFn
 import org.apache.gearpump.streaming.sink.{DataSink, DataSinkProcessor}
 import org.apache.gearpump.streaming.source.{DataSource, DataSourceTask}
 import org.apache.gearpump.streaming.task.Task
@@ -130,12 +129,11 @@
 
   override def description: String = fn.description
 
-
   override def chain(other: Op)(implicit system: ActorSystem): Op = {
     other match {
       case op: ChainableOp[OUT, _] =>
         // TODO: preserve type info
-        ChainableOp(fn.andThen(op.fn))
+        ChainableOp(AndThen(fn, op.fn))
       case _ =>
         throw new OpChainException(this, other)
     }
@@ -147,15 +145,6 @@
   }
 }
 
-object GroupByOp {
-
-  def apply[IN, GROUP](groupBy: IN => GROUP, parallelism: Int,
-      description: String, userConfig: UserConfig): Op = {
-    GroupByOp(GroupAlsoByWindow(groupBy, CountWindow.apply(1).accumulating), parallelism,
-      description, userConfig)
-  }
-}
-
 /**
  * This represents a Processor with window aggregation
  */
diff --git a/streaming/src/main/scala/org/apache/gearpump/streaming/dsl/plan/Planner.scala b/streaming/src/main/scala/org/apache/gearpump/streaming/dsl/plan/Planner.scala
index 16d5c06..65f9cd2 100644
--- a/streaming/src/main/scala/org/apache/gearpump/streaming/dsl/plan/Planner.scala
+++ b/streaming/src/main/scala/org/apache/gearpump/streaming/dsl/plan/Planner.scala
@@ -20,7 +20,7 @@
 
 import akka.actor.ActorSystem
 
-import org.apache.gearpump.partitioner.{CoLocationPartitioner, HashPartitioner, Partitioner}
+import org.apache.gearpump.streaming.partitioner.{CoLocationPartitioner, HashPartitioner, Partitioner}
 import org.apache.gearpump.streaming.Processor
 import org.apache.gearpump.streaming.dsl.partitioner.GroupByPartitioner
 import org.apache.gearpump.streaming.task.Task
diff --git a/streaming/src/main/scala/org/apache/gearpump/streaming/dsl/plan/functions/SingleInputFunction.scala b/streaming/src/main/scala/org/apache/gearpump/streaming/dsl/plan/functions/SingleInputFunction.scala
index 609fbb0..687fd2e 100644
--- a/streaming/src/main/scala/org/apache/gearpump/streaming/dsl/plan/functions/SingleInputFunction.scala
+++ b/streaming/src/main/scala/org/apache/gearpump/streaming/dsl/plan/functions/SingleInputFunction.scala
@@ -17,20 +17,37 @@
  */
 package org.apache.gearpump.streaming.dsl.plan.functions
 
-trait SingleInputFunction[IN, OUT] extends Serializable {
+import org.apache.gearpump.streaming.dsl.api.functions.ReduceFunction
+import org.apache.gearpump.streaming.dsl.scalaapi.functions.FlatMapFunction
+
+/**
+ * Internal function to process single input
+ *
+ * @param IN input value type
+ * @param OUT output value type
+ */
+sealed trait SingleInputFunction[IN, OUT] extends java.io.Serializable {
+
+  def setup(): Unit = {}
+
   def process(value: IN): TraversableOnce[OUT]
-  def andThen[OUTER](other: SingleInputFunction[OUT, OUTER]): SingleInputFunction[IN, OUTER] = {
-    new AndThen(this, other)
-  }
+
   def finish(): TraversableOnce[OUT] = None
-  def clearState(): Unit = {}
+
+  def teardown(): Unit = {}
+
   def description: String
 }
 
-class AndThen[IN, MIDDLE, OUT](
-    first: SingleInputFunction[IN, MIDDLE], second: SingleInputFunction[MIDDLE, OUT])
+case class AndThen[IN, MIDDLE, OUT](first: SingleInputFunction[IN, MIDDLE],
+    second: SingleInputFunction[MIDDLE, OUT])
   extends SingleInputFunction[IN, OUT] {
 
+  override def setup(): Unit = {
+    first.setup()
+    second.setup()
+  }
+
   override def process(value: IN): TraversableOnce[OUT] = {
     first.process(value).flatMap(second.process)
   }
@@ -44,9 +61,9 @@
     }
   }
 
-  override def clearState(): Unit = {
-    first.clearState()
-    second.clearState()
+  override def teardown(): Unit = {
+    first.teardown()
+    second.teardown()
   }
 
   override def description: String = {
@@ -56,22 +73,31 @@
   }
 }
 
-class FlatMapFunction[IN, OUT](fn: IN => TraversableOnce[OUT], descriptionMessage: String)
+class FlatMapper[IN, OUT](fn: FlatMapFunction[IN, OUT], val description: String)
   extends SingleInputFunction[IN, OUT] {
 
+  override def setup(): Unit = {
+    fn.setup()
+  }
+
   override def process(value: IN): TraversableOnce[OUT] = {
     fn(value)
   }
 
-  override def description: String = descriptionMessage
+  override def teardown(): Unit = {
+    fn.teardown()
+  }
 }
 
-
-class ReduceFunction[T](fn: (T, T) => T, descriptionMessage: String)
+class Reducer[T](fn: ReduceFunction[T], val description: String)
   extends SingleInputFunction[T, T] {
 
   private var state: Option[T] = None
 
+  override def setup(): Unit = {
+    fn.setup()
+  }
+
   override def process(value: T): TraversableOnce[T] = {
     if (state.isEmpty) {
       state = Option(value)
@@ -85,23 +111,18 @@
     state
   }
 
-  override def clearState(): Unit = {
+  override def teardown(): Unit = {
     state = None
+    fn.teardown()
   }
-
-  override def description: String = descriptionMessage
 }
 
-class EmitFunction[T](emit: T => Unit) extends SingleInputFunction[T, Unit] {
+class Emit[T](emit: T => Unit) extends SingleInputFunction[T, Unit] {
 
   override def process(value: T): TraversableOnce[Unit] = {
     emit(value)
     None
   }
 
-  override def andThen[R](other: SingleInputFunction[Unit, R]): SingleInputFunction[T, R] = {
-    throw new UnsupportedOperationException("andThen is not supposed to be called on EmitFunction")
-  }
-
   override def description: String = ""
 }
diff --git a/streaming/src/main/scala/org/apache/gearpump/streaming/dsl/Stream.scala b/streaming/src/main/scala/org/apache/gearpump/streaming/dsl/scalaapi/Stream.scala
similarity index 72%
rename from streaming/src/main/scala/org/apache/gearpump/streaming/dsl/Stream.scala
rename to streaming/src/main/scala/org/apache/gearpump/streaming/dsl/scalaapi/Stream.scala
index 440a45e..430d795 100644
--- a/streaming/src/main/scala/org/apache/gearpump/streaming/dsl/Stream.scala
+++ b/streaming/src/main/scala/org/apache/gearpump/streaming/dsl/scalaapi/Stream.scala
@@ -16,14 +16,16 @@
  * limitations under the License.
  */
 
-package org.apache.gearpump.streaming.dsl
+package org.apache.gearpump.streaming.dsl.scalaapi
 
 import org.apache.gearpump.Message
 import org.apache.gearpump.cluster.UserConfig
+import org.apache.gearpump.streaming.dsl.api.functions.{FilterFunction, MapFunction, ReduceFunction}
+import org.apache.gearpump.streaming.dsl.scalaapi.functions.FlatMapFunction
 import org.apache.gearpump.streaming.dsl.plan._
-import org.apache.gearpump.streaming.dsl.plan.functions.{FlatMapFunction, ReduceFunction}
+import org.apache.gearpump.streaming.dsl.plan.functions._
 import org.apache.gearpump.streaming.dsl.window.api._
-import org.apache.gearpump.streaming.dsl.window.impl._
+import org.apache.gearpump.streaming.dsl.window.impl.{Bucket, GroupAlsoByWindow}
 import org.apache.gearpump.streaming.sink.DataSink
 import org.apache.gearpump.streaming.task.{Task, TaskContext}
 import org.apache.gearpump.util.Graph
@@ -36,55 +38,95 @@
     private val edge: Option[OpEdge] = None) {
 
   /**
-   * converts a value[T] to a list of value[R]
+   * Returns a new stream by applying a flatMap function to each element
+   * and flatten the results.
    *
-   * @param fn FlatMap function
+   * @param fn flatMap function
    * @param description The description message for this operation
    * @return A new stream with type [R]
    */
   def flatMap[R](fn: T => TraversableOnce[R], description: String = "flatMap"): Stream[R] = {
-    val flatMapOp = ChainableOp(new FlatMapFunction[T, R](fn, description))
-    graph.addVertex(flatMapOp)
-    graph.addEdge(thisNode, edge.getOrElse(Direct), flatMapOp)
-    new Stream[R](graph, flatMapOp)
+    this.flatMap(FlatMapFunction(fn), description)
   }
 
   /**
-   * Maps message of type T message of type R
+   * Returns a new stream by applying a flatMap function to each element
+   * and flatten the results.
    *
-   * @param fn Function
+   * @param fn flatMap function
+   * @param description The description message for this operation
+   * @return A new stream with type [R]
+   */
+  def flatMap[R](fn: FlatMapFunction[T, R], description: String): Stream[R] = {
+    transform(new FlatMapper[T, R](fn, description))
+  }
+
+  /**
+   * Returns a new stream by applying a map function to each element.
+   *
+   * @param fn map function
    * @return A new stream with type [R]
    */
   def map[R](fn: T => R, description: String = "map"): Stream[R] = {
-    this.flatMap({ data =>
-      Option(fn(data))
-    }, description)
+    this.map(MapFunction(fn), description)
   }
 
   /**
-   * Keeps records when fun(T) == true
+   * Returns a new stream by applying a map function to each element.
    *
-   * @param fn  the filter
-   * @return  a new stream after filter
+   * @param fn map function
+   * @return A new stream with type [R]
+   */
+  def map[R](fn: MapFunction[T, R], description: String): Stream[R] = {
+    this.flatMap(FlatMapFunction(fn), description)
+  }
+
+  /**
+   * Returns a new Stream keeping the elements that satisfy the filter function.
+   *
+   * @param fn filter function
+   * @return a new stream after filter
    */
   def filter(fn: T => Boolean, description: String = "filter"): Stream[T] = {
-    this.flatMap({ data =>
-      if (fn(data)) Option(data) else None
-    }, description)
+    this.filter(FilterFunction(fn), description)
   }
 
   /**
-   * Reduces operations.
+   * Returns a new Stream keeping the elements that satisfy the filter function.
    *
-   * @param fn  reduction function
+   * @param fn filter function
+   * @return a new stream after filter
+   */
+  def filter(fn: FilterFunction[T], description: String): Stream[T] = {
+    this.flatMap(FlatMapFunction(fn), description)
+  }
+  /**
+   * Returns a new stream by applying a reduce function over all the elements.
+   *
+   * @param fn reduce function
    * @param description description message for this operator
-   * @return a new stream after reduction
+   * @return a new stream after reduce
    */
   def reduce(fn: (T, T) => T, description: String = "reduce"): Stream[T] = {
-    val reduceOp = ChainableOp(new ReduceFunction(fn, description))
-    graph.addVertex(reduceOp)
-    graph.addEdge(thisNode, edge.getOrElse(Direct), reduceOp)
-    new Stream(graph, reduceOp)
+    reduce(ReduceFunction(fn), description)
+  }
+
+  /**
+   * Returns a new stream by applying a reduce function over all the elements.
+   *
+   * @param fn reduce function
+   * @param description description message for this operator
+   * @return a new stream after reduce
+   */
+  def reduce(fn: ReduceFunction[T], description: String): Stream[T] = {
+    transform(new Reducer[T](fn, description))
+  }
+
+  private def transform[R](fn: SingleInputFunction[T, R]): Stream[R] = {
+    val op = ChainableOp(fn)
+    graph.addVertex(op)
+    graph.addEdge(thisNode, edge.getOrElse(Direct), op)
+    new Stream(graph, op)
   }
 
   /**
diff --git a/streaming/src/main/scala/org/apache/gearpump/streaming/dsl/StreamApp.scala b/streaming/src/main/scala/org/apache/gearpump/streaming/dsl/scalaapi/StreamApp.scala
similarity index 98%
rename from streaming/src/main/scala/org/apache/gearpump/streaming/dsl/StreamApp.scala
rename to streaming/src/main/scala/org/apache/gearpump/streaming/dsl/scalaapi/StreamApp.scala
index 8116146..d6eed2e 100644
--- a/streaming/src/main/scala/org/apache/gearpump/streaming/dsl/StreamApp.scala
+++ b/streaming/src/main/scala/org/apache/gearpump/streaming/dsl/scalaapi/StreamApp.scala
@@ -16,11 +16,12 @@
  * limitations under the License.
  */
 
-package org.apache.gearpump.streaming.dsl
+package org.apache.gearpump.streaming.dsl.scalaapi
 
 import java.time.Instant
 
 import akka.actor.ActorSystem
+import org.apache.gearpump.Message
 import org.apache.gearpump.cluster.UserConfig
 import org.apache.gearpump.cluster.client.ClientContext
 import org.apache.gearpump.streaming.StreamApplication
@@ -28,7 +29,6 @@
 import org.apache.gearpump.streaming.source.DataSource
 import org.apache.gearpump.streaming.task.TaskContext
 import org.apache.gearpump.util.Graph
-import org.apache.gearpump.Message
 
 import scala.language.implicitConversions
 
diff --git a/streaming/src/main/scala/org/apache/gearpump/streaming/dsl/scalaapi/functions/FlatMapFunction.scala b/streaming/src/main/scala/org/apache/gearpump/streaming/dsl/scalaapi/functions/FlatMapFunction.scala
new file mode 100644
index 0000000..f10a3db
--- /dev/null
+++ b/streaming/src/main/scala/org/apache/gearpump/streaming/dsl/scalaapi/functions/FlatMapFunction.scala
@@ -0,0 +1,103 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES 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.gearpump.streaming.dsl.scalaapi.functions
+
+import org.apache.gearpump.streaming.dsl.api.functions.{FilterFunction, MapFunction}
+import org.apache.gearpump.streaming.dsl.javaapi.functions.{FlatMapFunction => JFlatMapFunction}
+
+import scala.collection.JavaConverters._
+
+object FlatMapFunction {
+
+  def apply[T, R](fn: JFlatMapFunction[T, R]): FlatMapFunction[T, R] = {
+    new FlatMapFunction[T, R] {
+
+      override def setup(): Unit = {
+        fn.setup()
+      }
+
+      override def apply(t: T): TraversableOnce[R] = {
+        fn.apply(t).asScala
+      }
+
+
+      override def teardown(): Unit = {
+        fn.teardown()
+      }
+    }
+  }
+
+  def apply[T, R](fn: T => TraversableOnce[R]): FlatMapFunction[T, R] = {
+    new FlatMapFunction[T, R] {
+      override def apply(t: T): TraversableOnce[R] = {
+        fn(t)
+      }
+    }
+  }
+
+  def apply[T, R](fn: MapFunction[T, R]): FlatMapFunction[T, R] = {
+    new FlatMapFunction[T, R] {
+
+      override def setup(): Unit = {
+        fn.setup()
+      }
+
+      override def apply(t: T): TraversableOnce[R] = {
+        Option(fn(t))
+      }
+
+      override def teardown(): Unit = {
+        fn.teardown()
+      }
+    }
+  }
+
+  def apply[T, R](fn: FilterFunction[T]): FlatMapFunction[T, T] = {
+    new FlatMapFunction[T, T] {
+
+      override def setup(): Unit = {
+        fn.setup()
+      }
+
+      override def apply(t: T): TraversableOnce[T] = {
+        if (fn(t)) {
+          Option(t)
+        } else {
+          None
+        }
+      }
+
+      override def teardown(): Unit = {
+        fn.teardown()
+      }
+    }
+  }
+}
+
+/**
+ * Transforms one input into zero or more outputs of possibly different types.
+ * This Scala version of FlatMapFunction returns a TraversableOnce.
+ *
+ * @param T Input value type
+ * @param R Output value type
+ */
+abstract class FlatMapFunction[T, R] extends SerializableFunction {
+
+  def apply(t: T): TraversableOnce[R]
+
+}
diff --git a/core/src/main/scala/org/apache/gearpump/partitioner/CoLocationPartitioner.scala b/streaming/src/main/scala/org/apache/gearpump/streaming/dsl/scalaapi/functions/SerializableFunction.scala
similarity index 65%
copy from core/src/main/scala/org/apache/gearpump/partitioner/CoLocationPartitioner.scala
copy to streaming/src/main/scala/org/apache/gearpump/streaming/dsl/scalaapi/functions/SerializableFunction.scala
index 5a3eec4..ab88bf1 100644
--- a/core/src/main/scala/org/apache/gearpump/partitioner/CoLocationPartitioner.scala
+++ b/streaming/src/main/scala/org/apache/gearpump/streaming/dsl/scalaapi/functions/SerializableFunction.scala
@@ -15,17 +15,18 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-
-package org.apache.gearpump.partitioner
-
-import org.apache.gearpump.Message
+package org.apache.gearpump.streaming.dsl.scalaapi.functions
 
 /**
- * Will have the same parallelism with last processor
- * And each task in current processor will co-locate with task of last processor
+ * Superclass for all user defined function interfaces.
+ * This ensures all functions are serializable and provides common methods
+ * like setup and teardown. Users should not extend this class directly
+ * but subclasses like [[FlatMapFunction]].
  */
-class CoLocationPartitioner extends UnicastPartitioner {
-  override def getPartition(msg: Message, partitionNum: Int, currentPartitionId: Int): Int = {
-    currentPartitionId
-  }
+abstract class SerializableFunction extends java.io.Serializable {
+
+  def setup(): Unit = {}
+
+  def teardown(): Unit = {}
+
 }
diff --git a/streaming/src/main/scala/org/apache/gearpump/streaming/dsl/task/CountTriggerTask.scala b/streaming/src/main/scala/org/apache/gearpump/streaming/dsl/task/CountTriggerTask.scala
index 4ee2fa8..06f2964 100644
--- a/streaming/src/main/scala/org/apache/gearpump/streaming/dsl/task/CountTriggerTask.scala
+++ b/streaming/src/main/scala/org/apache/gearpump/streaming/dsl/task/CountTriggerTask.scala
@@ -19,7 +19,6 @@
 
 import java.time.Instant
 
-import akka.actor.ActorSystem
 import org.apache.gearpump.Message
 import org.apache.gearpump.cluster.UserConfig
 import org.apache.gearpump.streaming.Constants._
diff --git a/streaming/src/main/scala/org/apache/gearpump/streaming/dsl/task/EventTimeTriggerTask.scala b/streaming/src/main/scala/org/apache/gearpump/streaming/dsl/task/EventTimeTriggerTask.scala
index 4b7649f..0674339 100644
--- a/streaming/src/main/scala/org/apache/gearpump/streaming/dsl/task/EventTimeTriggerTask.scala
+++ b/streaming/src/main/scala/org/apache/gearpump/streaming/dsl/task/EventTimeTriggerTask.scala
@@ -19,7 +19,6 @@
 
 import java.time.Instant
 
-import akka.actor.ActorSystem
 import org.apache.gearpump.Message
 import org.apache.gearpump.cluster.UserConfig
 import org.apache.gearpump.streaming.Constants._
diff --git a/streaming/src/main/scala/org/apache/gearpump/streaming/dsl/task/ProcessingTimeTriggerTask.scala b/streaming/src/main/scala/org/apache/gearpump/streaming/dsl/task/ProcessingTimeTriggerTask.scala
index 980a54b..78ba762 100644
--- a/streaming/src/main/scala/org/apache/gearpump/streaming/dsl/task/ProcessingTimeTriggerTask.scala
+++ b/streaming/src/main/scala/org/apache/gearpump/streaming/dsl/task/ProcessingTimeTriggerTask.scala
@@ -21,7 +21,6 @@
 import java.util.concurrent.TimeUnit
 
 import akka.actor.Actor.Receive
-import akka.actor.ActorSystem
 import org.apache.gearpump.Message
 import org.apache.gearpump.cluster.UserConfig
 import org.apache.gearpump.streaming.Constants._
diff --git a/streaming/src/main/scala/org/apache/gearpump/streaming/dsl/task/TransformTask.scala b/streaming/src/main/scala/org/apache/gearpump/streaming/dsl/task/TransformTask.scala
index e35f085..f8fbefa 100644
--- a/streaming/src/main/scala/org/apache/gearpump/streaming/dsl/task/TransformTask.scala
+++ b/streaming/src/main/scala/org/apache/gearpump/streaming/dsl/task/TransformTask.scala
@@ -17,21 +17,26 @@
  */
 package org.apache.gearpump.streaming.dsl.task
 
+import java.time.Instant
+
 import org.apache.gearpump.Message
 import org.apache.gearpump.cluster.UserConfig
 import org.apache.gearpump.streaming.Constants._
 import org.apache.gearpump.streaming.dsl.plan.functions.SingleInputFunction
 import org.apache.gearpump.streaming.task.{Task, TaskContext}
 
-class TransformTask[IN, OUT](
-    operator: Option[SingleInputFunction[IN, OUT]], taskContext: TaskContext,
-    userConf: UserConfig) extends Task(taskContext, userConf) {
+class TransformTask[IN, OUT](operator: Option[SingleInputFunction[IN, OUT]],
+    taskContext: TaskContext, userConf: UserConfig) extends Task(taskContext, userConf) {
 
   def this(taskContext: TaskContext, userConf: UserConfig) = {
     this(userConf.getValue[SingleInputFunction[IN, OUT]](
       GEARPUMP_STREAMING_OPERATOR)(taskContext.system), taskContext, userConf)
   }
 
+  override def onStart(startTime: Instant): Unit = {
+    operator.foreach(_.setup())
+  }
+
   override def onNext(msg: Message): Unit = {
     val time = msg.timestamp
 
@@ -44,4 +49,8 @@
         taskContext.output(new Message(msg.msg, time))
     }
   }
+
+  override def onStop(): Unit = {
+    operator.foreach(_.teardown())
+  }
 }
diff --git a/streaming/src/main/scala/org/apache/gearpump/streaming/dsl/window/impl/WindowRunner.scala b/streaming/src/main/scala/org/apache/gearpump/streaming/dsl/window/impl/WindowRunner.scala
index 9af5e61..223a4af 100644
--- a/streaming/src/main/scala/org/apache/gearpump/streaming/dsl/window/impl/WindowRunner.scala
+++ b/streaming/src/main/scala/org/apache/gearpump/streaming/dsl/window/impl/WindowRunner.scala
@@ -22,12 +22,13 @@
 import akka.actor.ActorSystem
 import org.apache.gearpump.Message
 import org.apache.gearpump.cluster.UserConfig
-import org.apache.gearpump.gs.collections.api.block.procedure.Procedure
-import org.apache.gearpump.gs.collections.impl.list.mutable.FastList
-import org.apache.gearpump.gs.collections.impl.map.mutable.UnifiedMap
-import org.apache.gearpump.gs.collections.impl.map.sorted.mutable.TreeSortedMap
+import com.gs.collections.api.block.procedure.Procedure
+import com.gs.collections.impl.list.mutable.FastList
+import com.gs.collections.impl.map.mutable.UnifiedMap
+import com.gs.collections.impl.map.sorted.mutable.TreeSortedMap
+import com.gs.collections.impl.set.mutable.UnifiedSet
 import org.apache.gearpump.streaming.Constants._
-import org.apache.gearpump.streaming.dsl.plan.functions.{EmitFunction, SingleInputFunction}
+import org.apache.gearpump.streaming.dsl.plan.functions.{AndThen, Emit, SingleInputFunction}
 import org.apache.gearpump.streaming.dsl.window.api.Discarding
 import org.apache.gearpump.streaming.task.TaskContext
 import org.apache.gearpump.util.LogUtil
@@ -38,7 +39,6 @@
   def process(message: Message): Unit
 
   def trigger(time: Instant): Unit
-
 }
 
 object DefaultWindowRunner {
@@ -46,18 +46,6 @@
   private val LOG: Logger = LogUtil.getLogger(classOf[DefaultWindowRunner[_, _, _]])
 
   case class WindowGroup[GROUP](bucket: Bucket, group: GROUP)
-    extends Comparable[WindowGroup[GROUP]] {
-    override def compareTo(o: WindowGroup[GROUP]): Int = {
-      val ret = bucket.compareTo(o.bucket)
-      if (ret != 0) {
-        ret
-      } else if (group.equals(o.group)) {
-        0
-      } else {
-        -1
-      }
-    }
-  }
 }
 
 class DefaultWindowRunner[IN, GROUP, OUT](
@@ -66,20 +54,27 @@
   extends WindowRunner {
   import org.apache.gearpump.streaming.dsl.window.impl.DefaultWindowRunner._
 
-  private val windowGroups = new TreeSortedMap[WindowGroup[GROUP], FastList[IN]]
+  private val windows = new TreeSortedMap[Bucket, UnifiedSet[WindowGroup[GROUP]]]
+  private val windowGroups = new UnifiedMap[WindowGroup[GROUP], FastList[IN]]
   private val groupFns = new UnifiedMap[GROUP, SingleInputFunction[IN, OUT]]
 
-
   override def process(message: Message): Unit = {
     val (group, buckets) = groupBy.groupBy(message)
     buckets.foreach { bucket =>
       val wg = WindowGroup(bucket, group)
+      val wgs = windows.getOrDefault(bucket, new UnifiedSet[WindowGroup[GROUP]](1))
+      wgs.add(wg)
+      windows.put(bucket, wgs)
+
       val inputs = windowGroups.getOrDefault(wg, new FastList[IN](1))
       inputs.add(message.msg.asInstanceOf[IN])
       windowGroups.put(wg, inputs)
     }
-    groupFns.putIfAbsent(group,
-      userConfig.getValue[SingleInputFunction[IN, OUT]](GEARPUMP_STREAMING_OPERATOR).get)
+    if (!groupFns.containsKey(group)) {
+      val fn = userConfig.getValue[SingleInputFunction[IN, OUT]](GEARPUMP_STREAMING_OPERATOR).get
+      fn.setup()
+      groupFns.put(group, fn)
+    }
   }
 
   override def trigger(time: Instant): Unit = {
@@ -87,21 +82,28 @@
 
     @annotation.tailrec
     def onTrigger(): Unit = {
-      if (windowGroups.notEmpty()) {
-        val first = windowGroups.firstKey
-        if (!time.isBefore(first.bucket.endTime)) {
-          val inputs = windowGroups.remove(first)
-          val reduceFn = groupFns.get(first.group)
-            .andThen[Unit](new EmitFunction[OUT](emitResult(_, time)))
-          inputs.forEach(new Procedure[IN] {
-            override def value(t: IN): Unit = {
-              reduceFn.process(t)
+      if (windows.notEmpty()) {
+        val first = windows.firstKey
+        if (!time.isBefore(first.endTime)) {
+          val wgs = windows.remove(first)
+          wgs.forEach(new Procedure[WindowGroup[GROUP]] {
+            override def value(each: WindowGroup[GROUP]): Unit = {
+              val inputs = windowGroups.remove(each)
+              val reduceFn = AndThen(groupFns.get(each.group), new Emit[OUT](emitResult(_, time)))
+              inputs.forEach(new Procedure[IN] {
+                override def value(t: IN): Unit = {
+                  // .toList forces eager evaluation
+                  reduceFn.process(t).toList
+                }
+              })
+              // .toList forces eager evaluation
+              reduceFn.finish().toList
+              if (groupBy.window.accumulationMode == Discarding) {
+                reduceFn.teardown()
+              }
             }
           })
-          reduceFn.finish()
-          if (groupBy.window.accumulationMode == Discarding) {
-            reduceFn.clearState()
-          }
+
           onTrigger()
         }
       }
diff --git a/streaming/src/main/scala/org/apache/gearpump/streaming/metrics/ProcessorAggregator.scala b/streaming/src/main/scala/org/apache/gearpump/streaming/metrics/ProcessorAggregator.scala
index d045def..8f8b7ab 100644
--- a/streaming/src/main/scala/org/apache/gearpump/streaming/metrics/ProcessorAggregator.scala
+++ b/streaming/src/main/scala/org/apache/gearpump/streaming/metrics/ProcessorAggregator.scala
@@ -20,12 +20,11 @@
 
 import java.util
 
+import com.google.common.collect.Iterators
 import com.typesafe.config.Config
-
 import org.apache.gearpump.TimeStamp
 import org.apache.gearpump.cluster.ClientToMaster.ReadOption
 import org.apache.gearpump.cluster.MasterToClient.HistoryMetricsItem
-import org.apache.gearpump.google.common.collect.Iterators
 import org.apache.gearpump.metrics.Metrics.{Histogram, Meter}
 import org.apache.gearpump.metrics.MetricsAggregator
 import org.apache.gearpump.streaming.metrics.ProcessorAggregator._
diff --git a/core/src/main/scala/org/apache/gearpump/partitioner/BroadcastPartitioner.scala b/streaming/src/main/scala/org/apache/gearpump/streaming/partitioner/BroadcastPartitioner.scala
similarity index 95%
rename from core/src/main/scala/org/apache/gearpump/partitioner/BroadcastPartitioner.scala
rename to streaming/src/main/scala/org/apache/gearpump/streaming/partitioner/BroadcastPartitioner.scala
index 99cbcb6..9b63e04 100644
--- a/core/src/main/scala/org/apache/gearpump/partitioner/BroadcastPartitioner.scala
+++ b/streaming/src/main/scala/org/apache/gearpump/streaming/partitioner/BroadcastPartitioner.scala
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.gearpump.partitioner
+package org.apache.gearpump.streaming.partitioner
 
 import org.apache.gearpump.Message
 
diff --git a/core/src/main/scala/org/apache/gearpump/partitioner/CoLocationPartitioner.scala b/streaming/src/main/scala/org/apache/gearpump/streaming/partitioner/CoLocationPartitioner.scala
similarity index 95%
rename from core/src/main/scala/org/apache/gearpump/partitioner/CoLocationPartitioner.scala
rename to streaming/src/main/scala/org/apache/gearpump/streaming/partitioner/CoLocationPartitioner.scala
index 5a3eec4..4cb1bad 100644
--- a/core/src/main/scala/org/apache/gearpump/partitioner/CoLocationPartitioner.scala
+++ b/streaming/src/main/scala/org/apache/gearpump/streaming/partitioner/CoLocationPartitioner.scala
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.gearpump.partitioner
+package org.apache.gearpump.streaming.partitioner
 
 import org.apache.gearpump.Message
 
diff --git a/core/src/main/scala/org/apache/gearpump/partitioner/HashPartitioner.scala b/streaming/src/main/scala/org/apache/gearpump/streaming/partitioner/HashPartitioner.scala
similarity index 95%
rename from core/src/main/scala/org/apache/gearpump/partitioner/HashPartitioner.scala
rename to streaming/src/main/scala/org/apache/gearpump/streaming/partitioner/HashPartitioner.scala
index ee684a9..6137705 100644
--- a/core/src/main/scala/org/apache/gearpump/partitioner/HashPartitioner.scala
+++ b/streaming/src/main/scala/org/apache/gearpump/streaming/partitioner/HashPartitioner.scala
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.gearpump.partitioner
+package org.apache.gearpump.streaming.partitioner
 
 import org.apache.gearpump.Message
 
diff --git a/core/src/main/scala/org/apache/gearpump/partitioner/Partitioner.scala b/streaming/src/main/scala/org/apache/gearpump/streaming/partitioner/Partitioner.scala
similarity index 98%
rename from core/src/main/scala/org/apache/gearpump/partitioner/Partitioner.scala
rename to streaming/src/main/scala/org/apache/gearpump/streaming/partitioner/Partitioner.scala
index d68fa65..f685cc9 100644
--- a/core/src/main/scala/org/apache/gearpump/partitioner/Partitioner.scala
+++ b/streaming/src/main/scala/org/apache/gearpump/streaming/partitioner/Partitioner.scala
@@ -16,14 +16,13 @@
  * limitations under the License.
  */
 
-package org.apache.gearpump.partitioner
-
-import scala.reflect.ClassTag
+package org.apache.gearpump.streaming.partitioner
 
 import org.apache.commons.lang.SerializationUtils
-
 import org.apache.gearpump.Message
 
+import scala.reflect.ClassTag
+
 /**
  * For processor chain: A (3 tasks) {@literal ->} B (3 tasks), partitioner decide how ONE task
  * of upstream processor A send to several tasks of downstream processor B.
diff --git a/core/src/main/scala/org/apache/gearpump/partitioner/ShuffleGroupingPartitioner.scala b/streaming/src/main/scala/org/apache/gearpump/streaming/partitioner/ShuffleGroupingPartitioner.scala
similarity index 96%
rename from core/src/main/scala/org/apache/gearpump/partitioner/ShuffleGroupingPartitioner.scala
rename to streaming/src/main/scala/org/apache/gearpump/streaming/partitioner/ShuffleGroupingPartitioner.scala
index 55ef614..1b223e0 100644
--- a/core/src/main/scala/org/apache/gearpump/partitioner/ShuffleGroupingPartitioner.scala
+++ b/streaming/src/main/scala/org/apache/gearpump/streaming/partitioner/ShuffleGroupingPartitioner.scala
@@ -16,12 +16,12 @@
  * limitations under the License.
  */
 
-package org.apache.gearpump.partitioner
-
-import scala.util.Random
+package org.apache.gearpump.streaming.partitioner
 
 import org.apache.gearpump.Message
 
+import scala.util.Random
+
 /**
  * The idea of ShuffleGroupingPartitioner is derived from Storm.
  * Messages are randomly distributed across the downstream's tasks in a way such that
diff --git a/core/src/main/scala/org/apache/gearpump/partitioner/ShufflePartitioner.scala b/streaming/src/main/scala/org/apache/gearpump/streaming/partitioner/ShufflePartitioner.scala
similarity index 96%
rename from core/src/main/scala/org/apache/gearpump/partitioner/ShufflePartitioner.scala
rename to streaming/src/main/scala/org/apache/gearpump/streaming/partitioner/ShufflePartitioner.scala
index 5c66d66..39d5e3b 100644
--- a/core/src/main/scala/org/apache/gearpump/partitioner/ShufflePartitioner.scala
+++ b/streaming/src/main/scala/org/apache/gearpump/streaming/partitioner/ShufflePartitioner.scala
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.gearpump.partitioner
+package org.apache.gearpump.streaming.partitioner
 
 import java.util.Random
 
diff --git a/streaming/src/main/scala/org/apache/gearpump/streaming/source/DataSourceTask.scala b/streaming/src/main/scala/org/apache/gearpump/streaming/source/DataSourceTask.scala
index 535497c..450f2d6 100644
--- a/streaming/src/main/scala/org/apache/gearpump/streaming/source/DataSourceTask.scala
+++ b/streaming/src/main/scala/org/apache/gearpump/streaming/source/DataSourceTask.scala
@@ -69,6 +69,7 @@
   override def onStart(startTime: Instant): Unit = {
     LOG.info(s"opening data source at $startTime")
     source.open(context, startTime)
+    operator.foreach(_.setup())
 
     self ! Watermark(source.getWatermark)
   }
@@ -82,6 +83,7 @@
   }
 
   override def onStop(): Unit = {
+    operator.foreach(_.teardown())
     LOG.info("closing data source...")
     source.close()
   }
diff --git a/streaming/src/main/scala/org/apache/gearpump/streaming/task/SerializerResolver.scala b/streaming/src/main/scala/org/apache/gearpump/streaming/task/SerializerResolver.scala
index 902c663..3b32163 100644
--- a/streaming/src/main/scala/org/apache/gearpump/streaming/task/SerializerResolver.scala
+++ b/streaming/src/main/scala/org/apache/gearpump/streaming/task/SerializerResolver.scala
@@ -18,7 +18,7 @@
 
 package org.apache.gearpump.streaming.task
 
-import org.apache.gearpump.esotericsoftware.kryo.util.{IntMap, ObjectMap}
+import com.esotericsoftware.kryo.util.{IntMap, ObjectMap}
 import org.apache.gearpump.streaming.task.SerializerResolver.Registration
 
 private[task] class SerializerResolver {
diff --git a/streaming/src/main/scala/org/apache/gearpump/streaming/task/Subscriber.scala b/streaming/src/main/scala/org/apache/gearpump/streaming/task/Subscriber.scala
index 692d7f9..5c99980 100644
--- a/streaming/src/main/scala/org/apache/gearpump/streaming/task/Subscriber.scala
+++ b/streaming/src/main/scala/org/apache/gearpump/streaming/task/Subscriber.scala
@@ -18,7 +18,7 @@
 
 package org.apache.gearpump.streaming.task
 
-import org.apache.gearpump.partitioner.PartitionerDescription
+import org.apache.gearpump.streaming.partitioner.PartitionerDescription
 import org.apache.gearpump.streaming.{DAG, LifeTime}
 
 /**
diff --git a/streaming/src/main/scala/org/apache/gearpump/streaming/task/Subscription.scala b/streaming/src/main/scala/org/apache/gearpump/streaming/task/Subscription.scala
index d9fbc82..4193fbf 100644
--- a/streaming/src/main/scala/org/apache/gearpump/streaming/task/Subscription.scala
+++ b/streaming/src/main/scala/org/apache/gearpump/streaming/task/Subscription.scala
@@ -20,8 +20,8 @@
 
 import org.slf4j.Logger
 
-import org.apache.gearpump.google.common.primitives.Shorts
-import org.apache.gearpump.partitioner.{MulticastPartitioner, Partitioner, UnicastPartitioner}
+import com.google.common.primitives.Shorts
+import org.apache.gearpump.streaming.partitioner.{MulticastPartitioner, Partitioner, UnicastPartitioner}
 import org.apache.gearpump.streaming.AppMasterToExecutor.MsgLostException
 import org.apache.gearpump.streaming.LifeTime
 import org.apache.gearpump.streaming.task.Subscription._
diff --git a/streaming/src/main/scala/org/apache/gearpump/streaming/task/TaskActor.scala b/streaming/src/main/scala/org/apache/gearpump/streaming/task/TaskActor.scala
index f72e5b8..92f6672 100644
--- a/streaming/src/main/scala/org/apache/gearpump/streaming/task/TaskActor.scala
+++ b/streaming/src/main/scala/org/apache/gearpump/streaming/task/TaskActor.scala
@@ -23,17 +23,17 @@
 import java.util.concurrent.TimeUnit
 
 import akka.actor._
+import com.gs.collections.impl.map.mutable.primitive.IntShortHashMap
+import org.apache.gearpump.streaming.source.Watermark
+import org.slf4j.Logger
 import org.apache.gearpump.cluster.UserConfig
-import org.apache.gearpump.gs.collections.impl.map.mutable.primitive.IntShortHashMap
 import org.apache.gearpump.metrics.Metrics
 import org.apache.gearpump.serializer.SerializationFramework
 import org.apache.gearpump.streaming.AppMasterToExecutor._
 import org.apache.gearpump.streaming.ExecutorToAppMaster._
 import org.apache.gearpump.streaming.ProcessorId
-import org.apache.gearpump.streaming.source.Watermark
 import org.apache.gearpump.util.{LogUtil, TimeOutScheduler}
 import org.apache.gearpump.{Message, TimeStamp}
-import org.slf4j.Logger
 
 /**
  *
@@ -52,9 +52,10 @@
 
   def serializerPool: SerializationFramework = inputSerializerPool
 
+  import taskContextData._
+
   import org.apache.gearpump.streaming.Constants._
   import org.apache.gearpump.streaming.task.TaskActor._
-  import taskContextData._
   val config = context.system.settings.config
 
   val LOG: Logger = LogUtil.getLogger(getClass, app = appId, executor = executorId, task = taskId)
@@ -75,9 +76,9 @@
   private var life = taskContextData.life
 
   // Latency probe
-  import context.dispatcher
-
   import scala.concurrent.duration._
+
+  import context.dispatcher
   final val LATENCY_PROBE_INTERVAL = FiniteDuration(1, TimeUnit.SECONDS)
 
   // Clock report interval
diff --git a/streaming/src/test/scala/org/apache/gearpump/streaming/DAGSpec.scala b/streaming/src/test/scala/org/apache/gearpump/streaming/DAGSpec.scala
index 5f4faee..ccda8f0 100644
--- a/streaming/src/test/scala/org/apache/gearpump/streaming/DAGSpec.scala
+++ b/streaming/src/test/scala/org/apache/gearpump/streaming/DAGSpec.scala
@@ -18,7 +18,7 @@
 
 package org.apache.gearpump.streaming
 
-import org.apache.gearpump.partitioner.PartitionerDescription
+import org.apache.gearpump.streaming.partitioner.PartitionerDescription
 import org.apache.gearpump.streaming.task.TaskId
 import org.apache.gearpump.util.Graph
 import org.apache.gearpump.util.Graph.Node
diff --git a/streaming/src/test/scala/org/apache/gearpump/streaming/appmaster/AppMasterSpec.scala b/streaming/src/test/scala/org/apache/gearpump/streaming/appmaster/AppMasterSpec.scala
index e461ae8..29dfc57 100644
--- a/streaming/src/test/scala/org/apache/gearpump/streaming/appmaster/AppMasterSpec.scala
+++ b/streaming/src/test/scala/org/apache/gearpump/streaming/appmaster/AppMasterSpec.scala
@@ -34,7 +34,7 @@
 import org.apache.gearpump.cluster.scheduler.{Resource, ResourceAllocation, ResourceRequest}
 import org.apache.gearpump.cluster.worker.WorkerId
 import org.apache.gearpump.jarstore.FilePath
-import org.apache.gearpump.partitioner.HashPartitioner
+import org.apache.gearpump.streaming.partitioner.HashPartitioner
 import org.apache.gearpump.streaming.AppMasterToExecutor.StopTask
 import org.apache.gearpump.streaming.ExecutorToAppMaster.{MessageLoss, UnRegisterTask}
 import org.apache.gearpump.streaming.appmaster.AppMaster.{TaskActorRef, LookupTaskActorRef}
diff --git a/streaming/src/test/scala/org/apache/gearpump/streaming/appmaster/ClockServiceSpec.scala b/streaming/src/test/scala/org/apache/gearpump/streaming/appmaster/ClockServiceSpec.scala
index d42fe6f..46175a4 100644
--- a/streaming/src/test/scala/org/apache/gearpump/streaming/appmaster/ClockServiceSpec.scala
+++ b/streaming/src/test/scala/org/apache/gearpump/streaming/appmaster/ClockServiceSpec.scala
@@ -20,7 +20,7 @@
 import akka.actor.{ActorSystem, Props}
 import akka.testkit.{ImplicitSender, TestKit, TestProbe}
 import org.apache.gearpump.cluster.{TestUtil, UserConfig}
-import org.apache.gearpump.partitioner.{HashPartitioner, Partitioner, PartitionerDescription}
+import org.apache.gearpump.streaming.partitioner.{HashPartitioner, Partitioner, PartitionerDescription}
 import org.apache.gearpump.streaming.appmaster.ClockService.{ChangeToNewDAG, ChangeToNewDAGSuccess, HealthChecker, ProcessorClock}
 import org.apache.gearpump.streaming.appmaster.ClockServiceSpec.Store
 import org.apache.gearpump.streaming.storage.AppDataStore
diff --git a/streaming/src/test/scala/org/apache/gearpump/streaming/appmaster/DagManagerSpec.scala b/streaming/src/test/scala/org/apache/gearpump/streaming/appmaster/DagManagerSpec.scala
index be3b3b7..adde927 100644
--- a/streaming/src/test/scala/org/apache/gearpump/streaming/appmaster/DagManagerSpec.scala
+++ b/streaming/src/test/scala/org/apache/gearpump/streaming/appmaster/DagManagerSpec.scala
@@ -22,7 +22,7 @@
 import akka.actor.{ActorSystem, Props}
 import akka.testkit.TestProbe
 import org.apache.gearpump.cluster.{TestUtil, UserConfig}
-import org.apache.gearpump.partitioner.{HashPartitioner, Partitioner}
+import org.apache.gearpump.streaming.partitioner.{HashPartitioner, Partitioner}
 import org.apache.gearpump.streaming.appmaster.DagManager.{DAGOperationFailed, DAGOperationSuccess, GetLatestDAG, GetTaskLaunchData, LatestDAG, NewDAGDeployed, ReplaceProcessor, TaskLaunchData, WatchChange}
 import org.apache.gearpump.streaming.task.{Subscriber, TaskActor}
 import org.apache.gearpump.streaming._
diff --git a/streaming/src/test/scala/org/apache/gearpump/streaming/appmaster/JarSchedulerSpec.scala b/streaming/src/test/scala/org/apache/gearpump/streaming/appmaster/JarSchedulerSpec.scala
index 5f6dd04..def9d44 100644
--- a/streaming/src/test/scala/org/apache/gearpump/streaming/appmaster/JarSchedulerSpec.scala
+++ b/streaming/src/test/scala/org/apache/gearpump/streaming/appmaster/JarSchedulerSpec.scala
@@ -22,7 +22,7 @@
 import org.apache.gearpump.cluster.worker.WorkerId
 import org.apache.gearpump.cluster.{AppJar, TestUtil}
 import org.apache.gearpump.jarstore.FilePath
-import org.apache.gearpump.partitioner.{HashPartitioner, Partitioner}
+import org.apache.gearpump.streaming.partitioner.{HashPartitioner, Partitioner}
 import org.apache.gearpump.streaming.appmaster.TaskSchedulerSpec.{TestTask1, TestTask2}
 import org.apache.gearpump.streaming.task.TaskId
 import org.apache.gearpump.streaming.{DAG, ProcessorDescription, _}
diff --git a/streaming/src/test/scala/org/apache/gearpump/streaming/appmaster/TaskManagerSpec.scala b/streaming/src/test/scala/org/apache/gearpump/streaming/appmaster/TaskManagerSpec.scala
index 54ecde1..bcf96e4 100644
--- a/streaming/src/test/scala/org/apache/gearpump/streaming/appmaster/TaskManagerSpec.scala
+++ b/streaming/src/test/scala/org/apache/gearpump/streaming/appmaster/TaskManagerSpec.scala
@@ -25,7 +25,7 @@
 import org.apache.gearpump.cluster.worker.WorkerId
 import org.apache.gearpump.cluster.{AppJar, TestUtil, UserConfig}
 import org.apache.gearpump.jarstore.FilePath
-import org.apache.gearpump.partitioner.{HashPartitioner, Partitioner, PartitionerDescription}
+import org.apache.gearpump.streaming.partitioner.{HashPartitioner, Partitioner, PartitionerDescription}
 import org.apache.gearpump.streaming.AppMasterToExecutor.{LaunchTasks, StartAllTasks, StartDynamicDag, TaskLocationsReady, TaskLocationsReceived, TaskRegistered}
 import org.apache.gearpump.streaming.ExecutorToAppMaster.RegisterTask
 import org.apache.gearpump.streaming.appmaster.AppMaster.AllocateResourceTimeOut
diff --git a/streaming/src/test/scala/org/apache/gearpump/streaming/appmaster/TaskSchedulerSpec.scala b/streaming/src/test/scala/org/apache/gearpump/streaming/appmaster/TaskSchedulerSpec.scala
index 864aa93..1bfde94 100644
--- a/streaming/src/test/scala/org/apache/gearpump/streaming/appmaster/TaskSchedulerSpec.scala
+++ b/streaming/src/test/scala/org/apache/gearpump/streaming/appmaster/TaskSchedulerSpec.scala
@@ -21,7 +21,7 @@
 import org.apache.gearpump.cluster.scheduler.{Relaxation, Resource, ResourceRequest}
 import org.apache.gearpump.cluster.worker.WorkerId
 import org.apache.gearpump.cluster.{TestUtil, UserConfig}
-import org.apache.gearpump.partitioner.{HashPartitioner, Partitioner}
+import org.apache.gearpump.streaming.partitioner.{HashPartitioner, Partitioner}
 import org.apache.gearpump.streaming.appmaster.TaskLocator.Localities
 import org.apache.gearpump.streaming.appmaster.TaskSchedulerSpec.{TestTask1, TestTask2}
 import org.apache.gearpump.streaming.task.{Task, TaskContext, TaskId}
diff --git a/streaming/src/test/scala/org/apache/gearpump/streaming/dsl/plan/OpSpec.scala b/streaming/src/test/scala/org/apache/gearpump/streaming/dsl/plan/OpSpec.scala
index bf52abc..f0920de 100644
--- a/streaming/src/test/scala/org/apache/gearpump/streaming/dsl/plan/OpSpec.scala
+++ b/streaming/src/test/scala/org/apache/gearpump/streaming/dsl/plan/OpSpec.scala
@@ -25,7 +25,8 @@
 import org.apache.gearpump.streaming.Processor
 import org.apache.gearpump.streaming.Processor.DefaultProcessor
 import org.apache.gearpump.streaming.dsl.plan.OpSpec.{AnySink, AnySource, AnyTask}
-import org.apache.gearpump.streaming.dsl.plan.functions.SingleInputFunction
+import org.apache.gearpump.streaming.dsl.plan.functions.{FlatMapper, SingleInputFunction}
+import org.apache.gearpump.streaming.dsl.scalaapi.functions.FlatMapFunction
 import org.apache.gearpump.streaming.dsl.window.api.GroupByFn
 import org.apache.gearpump.streaming.sink.DataSink
 import org.apache.gearpump.streaming.source.DataSource
@@ -145,7 +146,6 @@
 
       val chainedOp = chainableOp1.chain(chainableOp2)
 
-      verify(fn1).andThen(fn2)
       chainedOp shouldBe a[ChainableOp[_, _]]
 
       unchainableOps.foreach { op =>
@@ -155,12 +155,14 @@
       }
     }
 
-    "throw exception on getProcessor" in {
-      val fn1 = mock[SingleInputFunction[Any, Any]]
-      val chainableOp1 = ChainableOp[Any, Any](fn1)
-      intercept[UnsupportedOperationException] {
-        chainableOp1.getProcessor
-      }
+    "get Processor" in {
+      val fn = mock[FlatMapFunction[Any, Any]]
+      val flatMapper = new FlatMapper(fn, "flatMap")
+      val chainableOp = ChainableOp[Any, Any](flatMapper)
+
+      val processor = chainableOp.getProcessor
+      processor shouldBe a[Processor[_]]
+      processor.parallelism shouldBe 1
     }
   }
 
diff --git a/streaming/src/test/scala/org/apache/gearpump/streaming/dsl/plan/PlannerSpec.scala b/streaming/src/test/scala/org/apache/gearpump/streaming/dsl/plan/PlannerSpec.scala
index f8666ba..3f23fa9 100644
--- a/streaming/src/test/scala/org/apache/gearpump/streaming/dsl/plan/PlannerSpec.scala
+++ b/streaming/src/test/scala/org/apache/gearpump/streaming/dsl/plan/PlannerSpec.scala
@@ -23,10 +23,12 @@
 import akka.actor.ActorSystem
 import org.apache.gearpump.Message
 import org.apache.gearpump.cluster.{TestUtil, UserConfig}
-import org.apache.gearpump.partitioner.CoLocationPartitioner
+import org.apache.gearpump.streaming.dsl.api.functions.ReduceFunction
+import org.apache.gearpump.streaming.partitioner.CoLocationPartitioner
 import org.apache.gearpump.streaming.dsl.partitioner.GroupByPartitioner
 import org.apache.gearpump.streaming.dsl.plan.PlannerSpec._
-import org.apache.gearpump.streaming.dsl.plan.functions.{FlatMapFunction, ReduceFunction}
+import org.apache.gearpump.streaming.dsl.plan.functions.{FlatMapper, Reducer}
+import org.apache.gearpump.streaming.dsl.scalaapi.functions.FlatMapFunction
 import org.apache.gearpump.streaming.dsl.window.api.GroupByFn
 import org.apache.gearpump.streaming.sink.DataSink
 import org.apache.gearpump.streaming.source.DataSource
@@ -56,8 +58,8 @@
     val graph = Graph.empty[Op, OpEdge]
     val sourceOp = DataSourceOp(new AnySource)
     val groupByOp = GroupByOp(new AnyGroupByFn)
-    val flatMapOp = ChainableOp[Any, Any](anyFlatMapFunction)
-    val reduceOp = ChainableOp[Any, Any](anyReduceFunction)
+    val flatMapOp = ChainableOp[Any, Any](anyFlatMapper)
+    val reduceOp = ChainableOp[Any, Any](anyReducer)
     val processorOp = new ProcessorOp[AnyTask]
     val sinkOp = DataSinkOp(new AnySink)
     val directEdge = Direct
@@ -92,9 +94,10 @@
 object PlannerSpec {
 
   private val anyParallelism = 1
-  private val anyFlatMapFunction = new FlatMapFunction[Any, Any](Option(_), "flatMap")
-  private val anyReduceFunction = new ReduceFunction[Any](
-    (left: Any, right: Any) => (left, right), "reduce")
+  private val anyFlatMapper = new FlatMapper[Any, Any](
+    FlatMapFunction(Option(_)), "flatMap")
+  private val anyReducer = new Reducer[Any](
+    ReduceFunction((left: Any, right: Any) => (left, right)), "reduce")
 
   class AnyTask(context: TaskContext, config: UserConfig) extends Task(context, config)
 
diff --git a/streaming/src/test/scala/org/apache/gearpump/streaming/dsl/plan/functions/SingleInputFunctionSpec.scala b/streaming/src/test/scala/org/apache/gearpump/streaming/dsl/plan/functions/SingleInputFunctionSpec.scala
index 94feae4..2c03e1c 100644
--- a/streaming/src/test/scala/org/apache/gearpump/streaming/dsl/plan/functions/SingleInputFunctionSpec.scala
+++ b/streaming/src/test/scala/org/apache/gearpump/streaming/dsl/plan/functions/SingleInputFunctionSpec.scala
@@ -23,9 +23,11 @@
 import org.apache.gearpump.Message
 import org.apache.gearpump.cluster.{TestUtil, UserConfig}
 import org.apache.gearpump.streaming.MockUtil
-import org.apache.gearpump.streaming.dsl.CollectionDataSource
 import org.apache.gearpump.streaming.source.DataSourceTask
 import org.apache.gearpump.streaming.Constants._
+import org.apache.gearpump.streaming.dsl.api.functions.ReduceFunction
+import org.apache.gearpump.streaming.dsl.scalaapi.CollectionDataSource
+import org.apache.gearpump.streaming.dsl.scalaapi.functions.FlatMapFunction
 import org.apache.gearpump.streaming.dsl.task.{CountTriggerTask, TransformTask}
 import org.apache.gearpump.streaming.dsl.window.api.CountWindow
 import org.apache.gearpump.streaming.dsl.window.impl.GroupAlsoByWindow
@@ -45,7 +47,7 @@
 
     val first = mock[SingleInputFunction[R, S]]
     val second = mock[SingleInputFunction[S, T]]
-    val andThen = new AndThen(first, second)
+    val andThen = AndThen(first, second)
 
     "chain first and second functions when processing input value" in {
       val input = mock[R]
@@ -77,145 +79,31 @@
       andThen.finish().toList shouldBe List(secondResult)
     }
 
-    "clear both states on clearState" in {
-      andThen.clearState()
+    "set up both functions on setup" in {
+      andThen.setup()
 
-      verify(first).clearState()
-      verify(second).clearState()
+      verify(first).setup()
+      verify(second).setup()
     }
 
-    "return AndThen on andThen" in {
-      val third = mock[SingleInputFunction[T, Any]]
-      andThen.andThen[Any](third) shouldBe an [AndThen[_, _, _]]
-    }
-  }
+    "tear down both functions on teardown" in {
+      andThen.teardown()
 
-  "FlatMapFunction" should {
-
-    val flatMap = mock[R => TraversableOnce[S]]
-    val flatMapFunction = new FlatMapFunction[R, S](flatMap, "flatMap")
-
-    "call flatMap function when processing input value" in {
-      val input = mock[R]
-      flatMapFunction.process(input)
-      verify(flatMap).apply(input)
+      verify(first).teardown()
+      verify(second).teardown()
     }
 
-    "return passed in description" in {
-      flatMapFunction.description shouldBe "flatMap"
-    }
-
-    "return None on finish" in {
-      flatMapFunction.finish() shouldBe List.empty[S]
-    }
-
-    "do nothing on clearState" in {
-      flatMapFunction.clearState()
-      verifyZeroInteractions(flatMap)
-    }
-
-    "return AndThen on andThen" in {
-      val other = mock[SingleInputFunction[S, T]]
-      flatMapFunction.andThen[T](other) shouldBe an [AndThen[_, _, _]]
-    }
-  }
-
-  "ReduceFunction" should {
-
-
-    "call reduce function when processing input value" in {
-      val reduce = mock[(T, T) => T]
-      val reduceFunction = new ReduceFunction[T](reduce, "reduce")
-      val input1 = mock[T]
-      val input2 = mock[T]
-      val output = mock[T]
-
-      when(reduce.apply(input1, input2)).thenReturn(output, output)
-
-      reduceFunction.process(input1) shouldBe List.empty[T]
-      reduceFunction.process(input2) shouldBe List.empty[T]
-      reduceFunction.finish() shouldBe List(output)
-
-      reduceFunction.clearState()
-      reduceFunction.process(input1) shouldBe List.empty[T]
-      reduceFunction.clearState()
-      reduceFunction.process(input2) shouldBe List.empty[T]
-      reduceFunction.finish() shouldBe List(input2)
-    }
-
-    "return passed in description" in {
-      val reduce = mock[(T, T) => T]
-      val reduceFunction = new ReduceFunction[T](reduce, "reduce")
-      reduceFunction.description shouldBe "reduce"
-    }
-
-    "return None on finish" in {
-      val reduce = mock[(T, T) => T]
-      val reduceFunction = new ReduceFunction[T](reduce, "reduce")
-      reduceFunction.finish() shouldBe List.empty[T]
-    }
-
-    "do nothing on clearState" in {
-      val reduce = mock[(T, T) => T]
-      val reduceFunction = new ReduceFunction[T](reduce, "reduce")
-      reduceFunction.clearState()
-      verifyZeroInteractions(reduce)
-    }
-
-    "return AndThen on andThen" in {
-      val reduce = mock[(T, T) => T]
-      val reduceFunction = new ReduceFunction[T](reduce, "reduce")
-      val other = mock[SingleInputFunction[T, Any]]
-      reduceFunction.andThen[Any](other) shouldBe an[AndThen[_, _, _]]
-    }
-  }
-
-  "EmitFunction" should {
-
-    val emit = mock[T => Unit]
-    val emitFunction = new EmitFunction[T](emit)
-
-    "emit input value when processing input value" in {
-      val input = mock[T]
-
-      emitFunction.process(input) shouldBe List.empty[Unit]
-
-      verify(emit).apply(input)
-    }
-
-    "return empty description" in {
-      emitFunction.description shouldBe ""
-    }
-
-    "return None on finish" in {
-      emitFunction.finish() shouldBe List.empty[Unit]
-    }
-
-    "do nothing on clearState" in {
-      emitFunction.clearState()
-      verifyZeroInteractions(emit)
-    }
-
-    "throw exception on andThen" in {
-      val other = mock[SingleInputFunction[Unit, Any]]
-      intercept[UnsupportedOperationException] {
-        emitFunction.andThen(other)
-      }
-    }
-  }
-
-  "andThen" should {
     "chain multiple single input function" in {
-      val split = new FlatMapFunction[String, String](line => line.split("\\s"), "split")
+      val split = new FlatMapper[String, String](FlatMapFunction(_.split("\\s")), "split")
 
-      val filter = new FlatMapFunction[String, String](word =>
-        if (word.isEmpty) None else Some(word), "filter")
+      val filter = new FlatMapper[String, String](
+        FlatMapFunction(word => if (word.isEmpty) None else Some(word)), "filter")
 
-      val map = new FlatMapFunction[String, Int](word => Some(1), "map")
+      val map = new FlatMapper[String, Int](FlatMapFunction(word => Some(1)), "map")
 
-      val sum = new ReduceFunction[Int]({ (left, right) => left + right }, "sum")
+      val sum = new Reducer[Int](ReduceFunction({(left, right) => left + right}), "sum")
 
-      val all = split.andThen(filter).andThen(map).andThen(sum)
+      val all = AndThen(split, AndThen(filter, AndThen(map, sum)))
 
       assert(all.description == "split.filter.map.sum")
 
@@ -235,13 +123,130 @@
     }
   }
 
+  "FlatMapper" should {
+
+    val flatMapFunction = mock[FlatMapFunction[R, S]]
+    val flatMapper = new FlatMapper[R, S](flatMapFunction, "flatMap")
+
+    "call flatMap function when processing input value" in {
+      val input = mock[R]
+      flatMapper.process(input)
+      verify(flatMapFunction).apply(input)
+    }
+
+    "return passed in description" in {
+      flatMapper.description shouldBe "flatMap"
+    }
+
+    "return None on finish" in {
+      flatMapper.finish() shouldBe List.empty[S]
+    }
+
+    "set up FlatMapFunction on setup" in {
+      flatMapper.setup()
+
+      verify(flatMapFunction).setup()
+    }
+
+    "tear down FlatMapFunction on teardown" in {
+      flatMapper.teardown()
+
+      verify(flatMapFunction).teardown()
+    }
+  }
+
+  "ReduceFunction" should {
+
+    "call reduce function when processing input value" in {
+      val reduceFunction = mock[ReduceFunction[T]]
+      val reducer = new Reducer[T](reduceFunction, "reduce")
+      val input1 = mock[T]
+      val input2 = mock[T]
+      val output = mock[T]
+
+      when(reduceFunction.apply(input1, input2)).thenReturn(output, output)
+
+      reducer.process(input1) shouldBe List.empty[T]
+      reducer.process(input2) shouldBe List.empty[T]
+      reducer.finish() shouldBe List(output)
+
+      reducer.teardown()
+      reducer.process(input1) shouldBe List.empty[T]
+      reducer.teardown()
+      reducer.process(input2) shouldBe List.empty[T]
+      reducer.finish() shouldBe List(input2)
+    }
+
+    "return passed in description" in {
+      val reduceFunction = mock[ReduceFunction[T]]
+      val reducer = new Reducer[T](reduceFunction, "reduce")
+      reducer.description shouldBe "reduce"
+    }
+
+    "return None on finish" in {
+      val reduceFunction = mock[ReduceFunction[T]]
+      val reducer = new Reducer[T](reduceFunction, "reduce")
+      reducer.finish() shouldBe List.empty[T]
+    }
+
+    "set up reduce function on setup" in {
+      val reduceFunction = mock[ReduceFunction[T]]
+      val reducer = new Reducer[T](reduceFunction, "reduce")
+      reducer.setup()
+
+      verify(reduceFunction).setup()
+    }
+
+    "tear down reduce function on teardown" in {
+      val reduceFunction = mock[ReduceFunction[T]]
+      val reducer = new Reducer[T](reduceFunction, "reduce")
+      reducer.teardown()
+
+      verify(reduceFunction).teardown()
+    }
+  }
+
+  "Emit" should {
+
+    val emitFunction = mock[T => Unit]
+    val emit = new Emit[T](emitFunction)
+
+    "emit input value when processing input value" in {
+      val input = mock[T]
+
+      emit.process(input) shouldBe List.empty[Unit]
+
+      verify(emitFunction).apply(input)
+    }
+
+    "return empty description" in {
+      emit.description shouldBe ""
+    }
+
+    "return None on finish" in {
+      emit.finish() shouldBe List.empty[Unit]
+    }
+
+    "do nothing on setup" in {
+      emit.setup()
+
+      verifyZeroInteractions(emitFunction)
+    }
+
+    "do nothing on teardown" in {
+      emit.teardown()
+
+      verifyZeroInteractions(emitFunction)
+    }
+  }
+
   "Source" should {
     "iterate over input source and apply attached operator" in {
 
       val taskContext = MockUtil.mockTaskContext
       implicit val actorSystem = MockUtil.system
 
-      val data = "one two three".split("\\s")
+      val data = "one two three".split("\\s+")
       val dataSource = new CollectionDataSource[String](data)
       val conf = UserConfig.empty.withValue(GEARPUMP_STREAMING_SOURCE, dataSource)
 
@@ -257,7 +262,8 @@
 
       // Source with transformer
       val anotherTaskContext = MockUtil.mockTaskContext
-      val double = new FlatMapFunction[String, String](word => List(word, word), "double")
+      val double = new FlatMapper[String, String](FlatMapFunction(
+        word => List(word, word)), "double")
       val another = new DataSourceTask(anotherTaskContext,
         conf.withValue(GEARPUMP_STREAMING_OPERATOR, double))
       another.onStart(Instant.EPOCH)
@@ -275,9 +281,8 @@
 
       val data = "1 2  2  3 3  3"
 
-      val concat = new ReduceFunction[String]({ (left, right) =>
-        left + right
-      }, "concat")
+      val concat = new Reducer[String](ReduceFunction({ (left, right) =>
+        left + right}), "concat")
 
       implicit val system = ActorSystem("test", TestUtil.DEFAULT_CONFIG)
       val config = UserConfig.empty.withValue[SingleInputFunction[String, String]](
@@ -311,7 +316,8 @@
       // Source with transformer
       val taskContext = MockUtil.mockTaskContext
       val conf = UserConfig.empty
-      val double = new FlatMapFunction[String, String](word => List(word, word), "double")
+      val double = new FlatMapper[String, String](FlatMapFunction(
+        word => List(word, word)), "double")
       val task = new TransformTask[String, String](Some(double), taskContext, conf)
       task.onStart(Instant.EPOCH)
 
diff --git a/streaming/src/test/scala/org/apache/gearpump/streaming/dsl/StreamAppSpec.scala b/streaming/src/test/scala/org/apache/gearpump/streaming/dsl/scalaapi/StreamAppSpec.scala
similarity index 88%
rename from streaming/src/test/scala/org/apache/gearpump/streaming/dsl/StreamAppSpec.scala
rename to streaming/src/test/scala/org/apache/gearpump/streaming/dsl/scalaapi/StreamAppSpec.scala
index e0407ec..5b90a3e 100644
--- a/streaming/src/test/scala/org/apache/gearpump/streaming/dsl/StreamAppSpec.scala
+++ b/streaming/src/test/scala/org/apache/gearpump/streaming/dsl/scalaapi/StreamAppSpec.scala
@@ -16,14 +16,15 @@
  * limitations under the License.
  */
 
-package org.apache.gearpump.streaming.dsl
+package org.apache.gearpump.streaming.dsl.scalaapi
 
 import akka.actor.ActorSystem
 import org.apache.gearpump.cluster.TestUtil
 import org.apache.gearpump.cluster.client.ClientContext
-import org.apache.gearpump.partitioner.PartitionerDescription
-import org.apache.gearpump.streaming.{ProcessorDescription, StreamApplication}
+import org.apache.gearpump.streaming.dsl.scalaapi
+import org.apache.gearpump.streaming.partitioner.PartitionerDescription
 import org.apache.gearpump.streaming.source.DataSourceTask
+import org.apache.gearpump.streaming.{ProcessorDescription, StreamApplication}
 import org.apache.gearpump.util.Graph
 import org.mockito.Mockito.when
 import org.scalatest._
@@ -49,8 +50,8 @@
     when(context.system).thenReturn(system)
 
     val dsl = StreamApp("dsl", context)
-    dsl.source(List("A"), 2, "A") shouldBe a [Stream[_]]
-    dsl.source(List("B"), 3, "B") shouldBe a [Stream[_]]
+    dsl.source(List("A"), 2, "A") shouldBe a [scalaapi.Stream[_]]
+    dsl.source(List("B"), 3, "B") shouldBe a [scalaapi.Stream[_]]
 
     val application = dsl.plan()
     application shouldBe a [StreamApplication]
diff --git a/streaming/src/test/scala/org/apache/gearpump/streaming/dsl/StreamSpec.scala b/streaming/src/test/scala/org/apache/gearpump/streaming/dsl/scalaapi/StreamSpec.scala
similarity index 92%
rename from streaming/src/test/scala/org/apache/gearpump/streaming/dsl/StreamSpec.scala
rename to streaming/src/test/scala/org/apache/gearpump/streaming/dsl/scalaapi/StreamSpec.scala
index fdc721b..62a3bcb 100644
--- a/streaming/src/test/scala/org/apache/gearpump/streaming/dsl/StreamSpec.scala
+++ b/streaming/src/test/scala/org/apache/gearpump/streaming/dsl/scalaapi/StreamSpec.scala
@@ -16,19 +16,19 @@
  * limitations under the License.
  */
 
-package org.apache.gearpump.streaming.dsl
+package org.apache.gearpump.streaming.dsl.scalaapi
 
 import akka.actor._
 import org.apache.gearpump.Message
 import org.apache.gearpump.cluster.client.ClientContext
 import org.apache.gearpump.cluster.{TestUtil, UserConfig}
-import org.apache.gearpump.partitioner.{CoLocationPartitioner, HashPartitioner, PartitionerDescription}
-import org.apache.gearpump.streaming.{ProcessorDescription, StreamApplication}
-import org.apache.gearpump.streaming.dsl.StreamSpec.Join
 import org.apache.gearpump.streaming.dsl.partitioner.GroupByPartitioner
+import org.apache.gearpump.streaming.dsl.scalaapi.StreamSpec.Join
 import org.apache.gearpump.streaming.dsl.task.{CountTriggerTask, TransformTask}
+import org.apache.gearpump.streaming.partitioner.{CoLocationPartitioner, HashPartitioner, PartitionerDescription}
 import org.apache.gearpump.streaming.source.DataSourceTask
 import org.apache.gearpump.streaming.task.{Task, TaskContext}
+import org.apache.gearpump.streaming.{ProcessorDescription, StreamApplication}
 import org.apache.gearpump.util.Graph
 import org.apache.gearpump.util.Graph._
 import org.mockito.Mockito.when
@@ -71,9 +71,10 @@
       map(word => (word, 1)).
       groupBy(_._1, parallelism = 2).
       reduce((left, right) => (left._1, left._2 + right._2)).
-      map[Either[(String, Int), String]](Left(_))
+      map[Either[(String, Int), String]]({t: (String, Int) => Left(t)})
 
-    val query = dsl.source(List("two"), 1, "").map[Either[(String, Int), String]](Right(_))
+    val query = dsl.source(List("two"), 1, "").map[Either[(String, Int), String]](
+      {s: String => Right(s)})
     stream.merge(query).process[(String, Int)](classOf[Join], 1)
 
     val app: StreamApplication = dsl.plan()
diff --git a/streaming/src/test/scala/org/apache/gearpump/streaming/dsl/task/TransformTaskSpec.scala b/streaming/src/test/scala/org/apache/gearpump/streaming/dsl/task/TransformTaskSpec.scala
new file mode 100644
index 0000000..b6e7342
--- /dev/null
+++ b/streaming/src/test/scala/org/apache/gearpump/streaming/dsl/task/TransformTaskSpec.scala
@@ -0,0 +1,75 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES 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.gearpump.streaming.dsl.task
+
+import java.time.Instant
+
+import org.apache.gearpump.Message
+import org.apache.gearpump.cluster.UserConfig
+import org.apache.gearpump.streaming.MockUtil
+import org.apache.gearpump.streaming.dsl.plan.functions.SingleInputFunction
+import org.mockito.Mockito.{verify, when}
+import org.scalacheck.Gen
+import org.scalatest.{Matchers, PropSpec}
+import org.scalatest.mock.MockitoSugar
+import org.scalatest.prop.PropertyChecks
+
+class TransformTaskSpec extends PropSpec with PropertyChecks with Matchers with MockitoSugar {
+
+  property("TransformTask.onStart should call SingleInputFunction.setup") {
+    forAll(Gen.chooseNum[Long](0L, 1000L).map(Instant.ofEpochMilli)) { (startTime: Instant) =>
+      val taskContext = MockUtil.mockTaskContext
+      implicit val system = MockUtil.system
+      val config = UserConfig.empty
+      val operator = mock[SingleInputFunction[Any, Any]]
+      val sourceTask = new TransformTask[Any, Any](Some(operator), taskContext, config)
+
+      sourceTask.onStart(startTime)
+
+      verify(operator).setup()
+    }
+  }
+
+  property("TransformTask.onNext should call SingleInputFunction.process") {
+    forAll(Gen.alphaStr) { (str: String) =>
+      val taskContext = MockUtil.mockTaskContext
+      implicit val system = MockUtil.system
+      val config = UserConfig.empty
+      val operator = mock[SingleInputFunction[Any, Any]]
+      val task = new TransformTask[Any, Any](Some(operator), taskContext, config)
+      val msg = Message(str)
+      when(operator.process(str)).thenReturn(Some(str))
+
+      task.onNext(msg)
+
+      verify(taskContext).output(msg)
+    }
+  }
+
+  property("DataSourceTask.onStop should call SingleInputFunction.setup") {
+    val taskContext = MockUtil.mockTaskContext
+    implicit val system = MockUtil.system
+    val config = UserConfig.empty
+    val operator = mock[SingleInputFunction[Any, Any]]
+    val task = new TransformTask[Any, Any](Some(operator), taskContext, config)
+
+    task.onStop()
+
+    verify(operator).teardown()
+  }
+}
diff --git a/core/src/test/scala/org/apache/gearpump/partitioner/PartitionerSpec.scala b/streaming/src/test/scala/org/apache/gearpump/streaming/partitioner/PartitionerSpec.scala
similarity index 97%
rename from core/src/test/scala/org/apache/gearpump/partitioner/PartitionerSpec.scala
rename to streaming/src/test/scala/org/apache/gearpump/streaming/partitioner/PartitionerSpec.scala
index fcf819b..277a31c 100644
--- a/core/src/test/scala/org/apache/gearpump/partitioner/PartitionerSpec.scala
+++ b/streaming/src/test/scala/org/apache/gearpump/streaming/partitioner/PartitionerSpec.scala
@@ -16,11 +16,10 @@
  * limitations under the License.
  */
 
-package org.apache.gearpump.partitioner
-
-import org.scalatest.{FlatSpec, Matchers}
+package org.apache.gearpump.streaming.partitioner
 
 import org.apache.gearpump.Message
+import org.scalatest.{FlatSpec, Matchers}
 
 class PartitionerSpec extends FlatSpec with Matchers {
   val NUM = 10
diff --git a/streaming/src/test/scala/org/apache/gearpump/streaming/source/DataSourceTaskSpec.scala b/streaming/src/test/scala/org/apache/gearpump/streaming/source/DataSourceTaskSpec.scala
index c786047..4e95bdd 100644
--- a/streaming/src/test/scala/org/apache/gearpump/streaming/source/DataSourceTaskSpec.scala
+++ b/streaming/src/test/scala/org/apache/gearpump/streaming/source/DataSourceTaskSpec.scala
@@ -23,6 +23,7 @@
 import org.apache.gearpump.Message
 import org.apache.gearpump.cluster.UserConfig
 import org.apache.gearpump.streaming.MockUtil
+import org.apache.gearpump.streaming.dsl.plan.functions.SingleInputFunction
 import org.mockito.Mockito._
 import org.scalacheck.Gen
 import org.scalatest.mock.MockitoSugar
@@ -38,11 +39,13 @@
       val dataSource = mock[DataSource]
       val config = UserConfig.empty
         .withInt(DataSourceConfig.SOURCE_READ_BATCH_SIZE, 1)
-
-      val sourceTask = new DataSourceTask[Any, Any](taskContext, config, dataSource, None)
+      val operator = mock[SingleInputFunction[Any, Any]]
+      val sourceTask = new DataSourceTask[Any, Any](taskContext, config, dataSource, Some(operator))
 
       sourceTask.onStart(startTime)
+
       verify(dataSource).open(taskContext, startTime)
+      verify(operator).setup()
     }
   }
 
@@ -69,9 +72,12 @@
     val dataSource = mock[DataSource]
     val config = UserConfig.empty
       .withInt(DataSourceConfig.SOURCE_READ_BATCH_SIZE, 1)
-    val sourceTask = new DataSourceTask[Any, Any](taskContext, config, dataSource, None)
+    val operator = mock[SingleInputFunction[Any, Any]]
+    val sourceTask = new DataSourceTask[Any, Any](taskContext, config, dataSource, Some(operator))
 
     sourceTask.onStop()
+
     verify(dataSource).close()
+    verify(operator).teardown()
   }
 }
diff --git a/streaming/src/test/scala/org/apache/gearpump/streaming/task/SubscriberSpec.scala b/streaming/src/test/scala/org/apache/gearpump/streaming/task/SubscriberSpec.scala
index cfe47eb..bd3ddb8 100644
--- a/streaming/src/test/scala/org/apache/gearpump/streaming/task/SubscriberSpec.scala
+++ b/streaming/src/test/scala/org/apache/gearpump/streaming/task/SubscriberSpec.scala
@@ -19,7 +19,7 @@
 
 import org.scalatest.{FlatSpec, Matchers}
 
-import org.apache.gearpump.partitioner.{HashPartitioner, Partitioner}
+import org.apache.gearpump.streaming.partitioner.{HashPartitioner, Partitioner}
 import org.apache.gearpump.streaming.task.SubscriberSpec.TestTask
 import org.apache.gearpump.streaming.{DAG, ProcessorDescription}
 import org.apache.gearpump.util.Graph
diff --git a/streaming/src/test/scala/org/apache/gearpump/streaming/task/SubscriptionSpec.scala b/streaming/src/test/scala/org/apache/gearpump/streaming/task/SubscriptionSpec.scala
index 258a5ff..d128ace 100644
--- a/streaming/src/test/scala/org/apache/gearpump/streaming/task/SubscriptionSpec.scala
+++ b/streaming/src/test/scala/org/apache/gearpump/streaming/task/SubscriptionSpec.scala
@@ -26,7 +26,7 @@
 
 import org.apache.gearpump.Message
 import org.apache.gearpump.cluster.UserConfig
-import org.apache.gearpump.partitioner.{HashPartitioner, Partitioner}
+import org.apache.gearpump.streaming.partitioner.{HashPartitioner, Partitioner}
 import org.apache.gearpump.streaming.task.SubscriptionSpec.NextTask
 import org.apache.gearpump.streaming.{LifeTime, ProcessorDescription}
 
diff --git a/streaming/src/test/scala/org/apache/gearpump/streaming/task/TaskActorSpec.scala b/streaming/src/test/scala/org/apache/gearpump/streaming/task/TaskActorSpec.scala
index 48901d2..8deee78 100644
--- a/streaming/src/test/scala/org/apache/gearpump/streaming/task/TaskActorSpec.scala
+++ b/streaming/src/test/scala/org/apache/gearpump/streaming/task/TaskActorSpec.scala
@@ -25,7 +25,7 @@
 
 import org.apache.gearpump.Message
 import org.apache.gearpump.cluster.{MasterHarness, TestUtil, UserConfig}
-import org.apache.gearpump.partitioner.{HashPartitioner, Partitioner}
+import org.apache.gearpump.streaming.partitioner.{HashPartitioner, Partitioner}
 import org.apache.gearpump.serializer.{FastKryoSerializer, SerializationFramework}
 import org.apache.gearpump.streaming.AppMasterToExecutor.{ChangeTask, MsgLostException, StartTask, TaskChanged, TaskRegistered}
 import org.apache.gearpump.streaming.task.TaskActorSpec.TestTask
diff --git a/version.sbt b/version.sbt
index 6f96fe0..f817483 100644
--- a/version.sbt
+++ b/version.sbt
@@ -16,4 +16,4 @@
  * limitations under the License.
  */
 
-version in ThisBuild := "0.8.2-SNAPSHOT"
+version in ThisBuild := "0.8.3-SNAPSHOT"