You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@zeppelin.apache.org by zj...@apache.org on 2019/11/19 06:51:20 UTC

[zeppelin] branch master updated: [ZEPPELIN-4323] Kotlin support for Spark interpreter

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

zjffdu pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/zeppelin.git


The following commit(s) were added to refs/heads/master by this push:
     new ee2613f  [ZEPPELIN-4323] Kotlin support for Spark interpreter
ee2613f is described below

commit ee2613fb25fcd033b558d1a021b43a9075faae34
Author: dkaznacheev <dm...@gmail.com>
AuthorDate: Mon Nov 18 16:22:37 2019 +0300

    [ZEPPELIN-4323] Kotlin support for Spark interpreter
    
    ### What is this PR for?
    This PR aims to add **Kotlin** language support to Apache Zeppelin as a standalone interpreter as well as a part of Spark interpreter group.
    For added features and details, see Jira issue: https://issues.apache.org/jira/browse/ZEPPELIN-4323
    
    Feel free to ask any questions, I will be happy to answer!
    
    ### What type of PR is it?
    Feature
    
    ### Todos
    * [ ] - Add support for Spark versions older than 2.4
    
    ### What is the Jira issue?
    https://issues.apache.org/jira/browse/ZEPPELIN-4323
    
    ### How should this be tested?
    * Unit tests are added to standalone Kotlin interpreter and to Spark interpreter.
    * Manually test by building Zepppelin under profile `-Pspark2.4` and using `%kotlin` in standalone interpreter or `%spark.kotlin` in Spark interpreter
    
    ### Screenshots (if appropriate)
    <img width="1680" alt="Screenshot 2019-09-06 at 14 58 00" src="https://user-images.githubusercontent.com/12711194/64426911-85de2980-d0b8-11e9-8311-d050284bb509.png">
    
    ### Questions:
    * Does the licenses files need update? No
    * Is there breaking changes for older versions? No
    * Does this needs documentation? Yes, and documentation is updated for Kotlin and Spark
    
    Author: dkaznacheev <dm...@gmail.com>
    Author: Ilya Muradyan <il...@jetbrains.com>
    Author: vitaly.khudobakhshov <vi...@gmail.com>
    
    Closes #3440 from dkaznacheev/kotlin-spark-interpreter and squashes the following commits:
    
    ad0380f3f [Ilya Muradyan] Changed hadoop-lzo repo and added Kotlin test
    9d5b6a3d7 [Ilya Muradyan] Temporary fix for hadoop_lzo
    ad2c1a916 [Ilya Muradyan] Increased memory limits
    c8e68e3dd [Ilya Muradyan] Merge branch 'master' into kotlin-spark-interpreter
    435808489 [Ilya Muradyan] Include kotlin interpreter in zeppelin_server builds
    347c9ebfd [Ilya Muradyan] Fix KotlinSpark interpreter test for old versions of spark.
    c8ebf7937 [Ilya Muradyan] [ZEPPELIN-4323] Merge branch 'refs/heads/master' into kotlin-spark-interpreter
    29a4dba19 [Ilya Muradyan] Merge branch 'refs/heads/master' into kotlin-spark-interpreter
    7a5252d13 [Ilya Muradyan] Fix comment indentation
    92f00af1d [Ilya Muradyan] Fix comment
    eacb964e1 [dkaznacheev] Added a comment about repl history records
    6e10ed716 [Ilya Muradyan] Merge branch 'refs/heads/master' into kotlin-spark-interpreter
    298457715 [dkaznacheev] Error fixes in classpath and type printing
    171a35be3 [dkaznacheev] Minor PR fixes
    2c25a7d80 [dkaznacheev] Minor PR fixes
    476fa376b [dkaznacheev] Fixed spark version checking
    61f666b33 [dkaznacheev] Added guide to testing and possible improvements
    54eebe359 [dkaznacheev] Added README.md for Kotlin interpreter
    065f0e251 [dkaznacheev] Various PR fixes
    deb152e18 [dkaznacheev] Wrapped output redirection in try/finally
    db495e873 [dkaznacheev] Removed unneccessary scala dependency
    b444001f7 [dkaznacheev] updated LICENSE
    ecd1f5d84 [dkaznacheev] Disabled Spark versions older than 2.4 to avoid build issues
    140265c84 [dkaznacheev] Fixing checkstyle violations
    8fc1be8b6 [dkaznacheev] Added Kotlin interpreter to Travis testing
    cc12fe8cb [dkaznacheev] Fixed bug with duplicate functions
    3120f1ada [dkaznacheev] Updated Zeppelin documentation
    2b42a6f36 [dkaznacheev] Fixed beam module not working
    79e203686 [dkaznacheev] Fixed type shortening in showFunctions()
    1269ee6f7 [dkaznacheev] refactored KotlinRepl constructor
    5c170f0fd [dkaznacheev] Added comments for Kotlin REPL
    1206df394 [dkaznacheev] Refactored REPL creation and properties setting
    c6c400df9 [dkaznacheev] Moved kotlin REPL to separate package
    8d9e9352a [dkaznacheev] Refactored REPL evaluation
    a08aa6bee [dkaznacheev] Moved class output to separate class
    26fac22a3 [dkaznacheev] Fixed completion shortening
    78426cb41 [dkaznacheev] Fixed interpreter settings
    251ea5af6 [dkaznacheev] Moved type shortening to an interpreter property
    ad91bcce9 [dkaznacheev] Implemented type shortening
    9979d032b [dkaznacheev] Refactored variables and methods search to separate class
    38bb29f33 [dkaznacheev] Added test for class writing in kotlin spark interpreter
    7c48cfacd [dkaznacheev] Moved KotlinReflectUtil to actual Kotlin types, updated tests
    cf488ca07 [dkaznacheev] (WIP) moved KotlinReflectUtil to actual Kotlin types
    0f95b1f31 [dkaznacheev] Added default classpath and maxResult to ReplBuilder
    9a359020c [dkaznacheev] Migrated to kotlin 1.3.50 from maven-central: updated tests
    60ed6be21 [dkaznacheev] Fixed tests in KotlinSparkInterpreter
    163a11f16 [dkaznacheev] Fixed retrieving variables from receivers and their child classes
    9914ed61a [dkaznacheev] Fixed completion order
    b9584de63 [dkaznacheev] Fixed competion on first cell in kotlin
    85727cdaf [dkaznacheev] Syncing with old kotlin-spark-interpreter branch
    a7ee65c82 [dkaznacheev] Fixed failing on getting wrapper type
    b3a09e580 [dkaznacheev] Fixed variable updating and invoking wrapper from JvmReplEvaluator
    d3f705d87 [vitaly.khudobakhshov] InvokeWrapper problem
    3a89c9041 [dkaznacheev] Fixed imports loading
    e80f2137e [dkaznacheev] Refactored classpath setting
    26997ea34 [dkaznacheev] Added and updated Kotlin documentation
    8d1d09e10 [dkaznacheev] Added tests for output classes
    9f0eb828d [dkaznacheev] Skipping conversion of types with type parameters
    6f59e4be0 [dkaznacheev] Added Kotlin types to completion
    0bcdb0833 [dkaznacheev] Added Kotlin keywords to completion
    64fe5fea2 [dkaznacheev] Updated completion
    95f6a472b [dkaznacheev] Added method retrieving and optimized variable retrieving
    4565a9e9f [dkaznacheev] Kotlin implicit receiver refactoring
    87c1e142b [dkaznacheev] Passing variable list to implicit receiver
    0d5316b5c [dkaznacheev] Cleaned up getting variables in runtime
    54a322fab [dkaznacheev] Retrieving variables from runtime
    d30a1e406 [dkaznacheev] Importing java udf libraries on startup
    99f4342b7 [dkaznacheev] Fixed z.select for Kotlin
    375e17531 [dkaznacheev] Fixed BaseZeppelinContext compile error
    7cd7d3454 [dkaznacheev] Removed z from base kotlin interpreter
    f1c4188f7 [dkaznacheev] Fixed spark cancel test
    0cc65bfdf [dkaznacheev] Added spark job cancellation test
    e1e05c894 [dkaznacheev] Refactored in-memory class output
    040d00edb [dkaznacheev] (not pretty) Fixed not writing in-memory Kotlin classes to repl output directory
    6d30cf3d8 [dkaznacheev] Added tests for Kotlin Spark interpreter
    0b2ab05af [dkaznacheev] Added cancellation and runtime exception tests for Kotlin interpreter
    839361fca [dkaznacheev] Resolved dependency issues on test run
    0a71d93dc [dkaznacheev] Added output test
    8e7221efd [dkaznacheev] Updated tests for plain Kotlin interpreter
    fec0b9ba6 [dkaznacheev] Fixing kotlin zeppelin interpreter not working
    54bdbdd4d [dkaznacheev] setting spark scheduler pool for Kotlin
    fc88280c0 [dkaznacheev] Fixed spark job cancellation for kotlin interpreter
    7c9ea1663 [dkaznacheev] Added z.select binding for Kotlin
    7b2ed795f [dkaznacheev] Added initial code execution to KotlinRepl
    0b95f1f32 [dkaznacheev] Added Apache 2.0 license to project files
    21bb45823 [dkaznacheev] Refactored df show fix
    cbe0ed04c [dkaznacheev] Fixed dataframe show() and showSchema() not working
    e919e9219 [dkaznacheev] Fixed zeppelin dynamic forms overriding kotlin string interpolation
    6d48f6f81 [dkaznacheev] Added SQLContext to spark kotlin context
    64f5849d2 [dkaznacheev] Removed local mvn repo dependencies
    76eaf171e [dkaznacheev] Fixed max result output limit for Kotlin REPL
    060a82209 [dkaznacheev] Added max result output limit for Kotlin REPL
    de1cd14e5 [dkaznacheev] Refactoring: renamed KotlinContext and children to KotlinReceiver to avoid confusion with ZeppelinContext
    019a7c5a3 [dkaznacheev] Added ZeppelinContext to default Kotlin Interpreter
    4aaf0bb3e [dkaznacheev] Added kotlin interpreter to SparkZeppelinContext list of interpreters
    b9e9f7e7f [dkaznacheev] Refactored Kotlin execution context
    fb5a64b3a [dkaznacheev] Fixed interpreter output error
    62d87c1af [dkaznacheev] Fixed show in ZeppelinContext
    ea024b5a8 [dkaznacheev] Added ZeppelinContext to ExecutionContext
    d478dd1bf [dkaznacheev] Switched to local kotlin JARs with bugfixes
    d89dbe696 [dkaznacheev] Shading jpountz module
    04e86d1c3 [dkaznacheev] Added class writing to classoutputdir
    b9a3927b1 [dkaznacheev] Fixed REPLInterpreter not using builder
    eb00b5c32 [dkaznacheev] Passing compiler options to KotlinInterpreter
    2eeb2f6dd [dkaznacheev] Reordered imports for checkstyle
    d196552a9 [dkaznacheev] Attempt to add spark context to ExecutionContext
    01cd61ef8 [dkaznacheev] Updated KotlinInterpreter to use new API
    eed0d9451 [dkaznacheev] Fixed maven-enforcer package version error
    f135a06e7 [dkaznacheev] Added Kotlin Spark interpreter stub
    634b75749 [dkaznacheev] Added object binding to KotlinInterpreter
    6c01762f0 [dkaznacheev] Changed KotlinInterpreter to use new scripting API
    b29c44847 [dkaznacheev] Added bind method to Kotlin interpreter
    6add47d46 [dkaznacheev] Minor cleanups
    bc018c7b9 [dkaznacheev] Added tests
    d79df4d44 [dkaznacheev] Implemented kotlin repl diagnostic message holder
    422c76ded [dkaznacheev] Added output and cancellation
    4016576ac [dkaznacheev] Turned on Kotlin syntax highlighting
    2cca5c999 [dkaznacheev] Fixed kotlin interpreter, path to java and kotlin runtimes are hardcoded
    bf6825ee5 [dkaznacheev] Added kotlin REPL interpreter with IO stubs
    61ff6d3c6 [dkaznacheev] Loading compiler configuration
    881cb418b [dkaznacheev] Added kotlin interpreter stub
---
 .travis.yml                                        |     4 +-
 LICENSE                                            |     1 +
 bin/common.sh                                      |     2 +-
 conf/interpreter-list                              |     1 +
 docs/_includes/themes/zeppelin/_navigation.html    |     1 +
 docs/index.md                                      |     1 +
 docs/interpreter/kotlin.md                         |    79 +
 docs/interpreter/spark.md                          |     9 +-
 docs/usage/interpreter/installation.md             |     5 +
 kotlin/README.md                                   |    83 +
 kotlin/pom.xml                                     |   125 +
 .../apache/zeppelin/kotlin/KotlinInterpreter.java  |   189 +
 .../kotlin/completion/KotlinCompleter.java         |    68 +
 .../zeppelin/kotlin/completion/KotlinKeywords.java |   104 +
 .../zeppelin/kotlin/context/KotlinReceiver.java    |    37 +
 .../zeppelin/kotlin/reflect/ContextUpdater.java    |   160 +
 .../kotlin/reflect/KotlinFunctionInfo.java         |    69 +
 .../zeppelin/kotlin/reflect/KotlinReflectUtil.java |    37 +
 .../kotlin/reflect/KotlinVariableInfo.java         |    60 +
 .../apache/zeppelin/kotlin/repl/ClassWriter.java   |    91 +
 .../apache/zeppelin/kotlin/repl/KotlinRepl.java    |   272 +
 .../kotlin/repl/building/KotlinReplProperties.java |   129 +
 .../kotlin/repl/building/ReplBuilding.java         |   108 +
 kotlin/src/main/resources/interpreter-setting.json |    28 +
 .../zeppelin/kotlin/KotlinInterpreterTest.java     |   326 +
 pom.xml                                            |     1 +
 scalding/pom.xml                                   |     7 +
 spark/interpreter/pom.xml                          |     8 +-
 .../zeppelin/spark/KotlinSparkInterpreter.java     |   198 +
 .../spark/kotlin/KotlinZeppelinBindings.java       |    52 +
 .../zeppelin/spark/kotlin/SparkKotlinReceiver.java |    43 +
 .../src/main/resources/interpreter-setting.json    |    34 +
 .../zeppelin/spark/KotlinSparkInterpreterTest.java |   246 +
 .../zeppelin/spark/SparkZeppelinContext.scala      |    11 +-
 zeppelin-web/bower.json                            |     1 +
 zeppelin-web/karma.conf.js                         |     1 +
 zeppelin-web/package-lock.json                     | 11578 ++++++++++---------
 zeppelin-web/src/index.html                        |     1 +
 .../zeppelin/storage/LocalConfigStorage.java       |     3 +-
 39 files changed, 8467 insertions(+), 5706 deletions(-)

diff --git a/.travis.yml b/.travis.yml
index ca2fce4..6956c4a 100644
--- a/.travis.yml
+++ b/.travis.yml
@@ -106,13 +106,13 @@ matrix:
     - sudo: required
       jdk: "openjdk8"
       dist: xenial
-      env: BUILD_PLUGINS="true" PYTHON="3" SCALA_VER="2.11" PROFILE="-Pspark-2.4 -Pspark-scala-2.11 -Phadoop2 -Pintegration" SPARKR="true" BUILD_FLAG="install -DskipTests -DskipRat -am" TEST_FLAG="test -DskipRat -am" MODULES="-pl zeppelin-interpreter-integration,jdbc,zeppelin-web,spark/spark-dependencies,markdown" TEST_PROJECTS="-Dtest=ZeppelinSparkClusterTest24,SparkIntegrationTest24,JdbcIntegrationTest,org.apache.zeppelin.spark.* -DfailIfNoTests=false"
+      env: BUILD_PLUGINS="true" PYTHON="3" SCALA_VER="2.11" PROFILE="-Pspark-2.4 -Pspark-scala-2.11 -Phadoop2 -Pintegration" SPARKR="true" BUILD_FLAG="install -DskipTests -DskipRat -am" TEST_FLAG="test -DskipRat -am" MODULES="-pl zeppelin-interpreter-integration,jdbc,zeppelin-web,spark/spark-dependencies,markdown" TEST_PROJECTS="-Dtest=ZeppelinSparkClusterTest24,SparkIntegrationTest24,JdbcIntegrationTest,org.apache.zeppelin.spark.*,org.apache.zeppelin.kotlin.* -DfailIfNoTests=false"
 
     # ZeppelinSparkClusterTest24, SparkIntegrationTest24, JdbcIntegrationTest, Unit test of Spark 2.4 (Scala-2.12)
     - sudo: required
       jdk: "openjdk8"
       dist: xenial
-      env: BUILD_PLUGINS="true" PYTHON="3" SCALA_VER="2.12" PROFILE="-Pspark-2.4 -Pspark-scala-2.12 -Phadoop2 -Pintegration" SPARKR="true" BUILD_FLAG="install -DskipTests -DskipRat -am" TEST_FLAG="test -DskipRat -am" MODULES="-pl zeppelin-interpreter-integration,jdbc,zeppelin-web,spark/spark-dependencies,markdown" TEST_PROJECTS="-Dtest=ZeppelinSparkClusterTest24,SparkIntegrationTest24,JdbcIntegrationTest,org.apache.zeppelin.spark.* -DfailIfNoTests=false"
+      env: BUILD_PLUGINS="true" PYTHON="3" SCALA_VER="2.12" PROFILE="-Pspark-2.4 -Pspark-scala-2.12 -Phadoop2 -Pintegration" SPARKR="true" BUILD_FLAG="install -DskipTests -DskipRat -am" TEST_FLAG="test -DskipRat -am" MODULES="-pl zeppelin-interpreter-integration,jdbc,zeppelin-web,spark/spark-dependencies,markdown" TEST_PROJECTS="-Dtest=ZeppelinSparkClusterTest24,SparkIntegrationTest24,JdbcIntegrationTest,org.apache.zeppelin.spark.*,org.apache.zeppelin.kotlin.* -DfailIfNoTests=false"
 
     # ZeppelinSparkClusterTest23, SparkIntegrationTest23, Unit test of Spark 2.3 (Scala-2.11) and Unit test PythonInterpreter under python2
     - sudo: required
diff --git a/LICENSE b/LICENSE
index 121d2e1..add447e 100644
--- a/LICENSE
+++ b/LICENSE
@@ -262,6 +262,7 @@ The text of each license is also included at licenses/LICENSE-[project]-[version
     (Apache 2.0) Google Cloud Client Library for Java (https://github.com/GoogleCloudPlatform/google-cloud-java)
     (Apache 2.0) concurrentunit (https://github.com/jhalterman/concurrentunit)
     (Apache 2.0) Embedded MongoDB (https://github.com/flapdoodle-oss/de.flapdoodle.embed.mongo)
+    (Apache 2.0) Kotlin (https://github.com/JetBrains/kotlin)
 
 ========================================================================
 BSD 3-Clause licenses
diff --git a/bin/common.sh b/bin/common.sh
index 6447ec8..0ebae66 100644
--- a/bin/common.sh
+++ b/bin/common.sh
@@ -114,7 +114,7 @@ if [[ -z "${ZEPPELIN_MEM}" ]]; then
 fi
 
 if [[ -z "${ZEPPELIN_INTP_MEM}" ]]; then
-  export ZEPPELIN_INTP_MEM="-Xms1024m -Xmx1024m -XX:MaxPermSize=512m"
+  export ZEPPELIN_INTP_MEM="-Xms1024m -Xmx2048m -XX:MaxPermSize=512m"
 fi
 
 JAVA_OPTS+=" ${ZEPPELIN_JAVA_OPTS} -Dfile.encoding=${ZEPPELIN_ENCODING} ${ZEPPELIN_MEM}"
diff --git a/conf/interpreter-list b/conf/interpreter-list
index 05cb9ff..38c7d5e 100644
--- a/conf/interpreter-list
+++ b/conf/interpreter-list
@@ -32,6 +32,7 @@ hbase           org.apache.zeppelin:zeppelin-hbase:0.9.0                Hbase in
 ignite          org.apache.zeppelin:zeppelin-ignite:0.9.0               Ignite interpreter
 java            org.apache.zeppelin:zeppelin-java:0.9.0                 Java interpreter
 jdbc            org.apache.zeppelin:zeppelin-jdbc:0.9.0                 Jdbc interpreter
+kotlin          org.apache.zeppelin:zeppelin-kotlin:0.9.0               Kotlin interpreter
 kylin           org.apache.zeppelin:zeppelin-kylin:0.9.0                Kylin interpreter
 lens            org.apache.zeppelin:zeppelin-lens:0.9.0                 Lens interpreter
 livy            org.apache.zeppelin:zeppelin-livy:0.9.0                 Livy interpreter
diff --git a/docs/_includes/themes/zeppelin/_navigation.html b/docs/_includes/themes/zeppelin/_navigation.html
index 9b83369..68252ef 100644
--- a/docs/_includes/themes/zeppelin/_navigation.html
+++ b/docs/_includes/themes/zeppelin/_navigation.html
@@ -142,6 +142,7 @@
                 <li><a href="{{BASE_PATH}}/interpreter/hive.html">Hive</a></li>
                 <li><a href="{{BASE_PATH}}/interpreter/ignite.html">Ignite</a></li>
                 <li><a href="{{BASE_PATH}}/interpreter/java.html">Java</a></li>
+                <li><a href="{{BASE_PATH}}/interpreter/kotlin.html">Kotlin</a></li>
                 <li><a href="{{BASE_PATH}}/interpreter/kylin.html">Kylin</a></li>
                 <li><a href="{{BASE_PATH}}/interpreter/lens.html">Lens</a></li>
                 <li><a href="{{BASE_PATH}}/interpreter/livy.html">Livy</a></li>
diff --git a/docs/index.md b/docs/index.md
index ad53721..17e01c2 100644
--- a/docs/index.md
+++ b/docs/index.md
@@ -144,6 +144,7 @@ limitations under the License.
   * [Ignite](./interpreter/ignite.html)
   * [JDBC](./interpreter/jdbc.html)
   * [Kylin](./interpreter/kylin.html)
+  * [Kotlin](./interpreter/kotlin.html)
   * [Lens](./interpreter/lens.html)
   * [Livy](./interpreter/livy.html)
   * [Mahout](./interpreter/mahout.html)
diff --git a/docs/interpreter/kotlin.md b/docs/interpreter/kotlin.md
new file mode 100644
index 0000000..bba0817
--- /dev/null
+++ b/docs/interpreter/kotlin.md
@@ -0,0 +1,79 @@
+---
+layout: page
+title: "Kotlin interpreter in Apache Zeppelin"
+description: "Kotlin is a cross-platform, statically typed, general-purpose programming language with type inference."
+group: interpreter
+---
+<!--
+Licensed under the Apache License, Version 2.0 (the "License");
+you may not use this file except in compliance with the License.
+You may obtain a copy of the License at
+
+http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS,
+WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+See the License for the specific language governing permissions and
+limitations under the License.
+-->
+
+{% include JB/setup %}
+
+# Kotlin interpreter for Apache Zeppelin
+
+<div id="toc"></div>
+
+## Overview
+Kotlin is a cross-platform, statically typed, general-purpose programming language with type inference.
+It is designed to interoperate fully with Java, and the JVM version of its standard library depends on the Java Class Library, but type inference allows its syntax to be more concise.
+
+## Configuration
+<table class="table-configuration">
+  <tr>
+    <th>Name</th>
+    <th>Default</th>
+    <th>Description</th>
+  </tr>
+  <tr>
+    <td>zeppelin.kotlin.maxResult</td>
+    <td>1000</td>
+    <td>Max n
+  <tr>
+    <td>zeppelin.kotlin.shortenTypes</td>
+    <td>true</td>
+    <td>Display shortened types instead of full, e.g. Int vs kotlin.Int</td>
+  </tr>
+</table>
+
+## Example
+
+```kotlin
+%kotlin 
+
+fun square(n: Int): Int = n * n
+```
+
+## Kotlin Context
+Kotlin context is accessible via `kc` object bound to the interpreter. 
+It holds `vars` and `functions` fields that return all user-defined variables and functions present in the interpreter.
+You can also print variables or functions by calling `kc.showVars()` or `kc.showFunctions()`.
+
+### Example
+
+
+```kotlin
+fun square(n: Int): Int = n * n
+
+val greeter = { s: String -> println("Hello $s!") }
+val l = listOf("Drive", "to", "develop")
+
+kc.showVars()
+kc.showFunctions()
+```
+Output:
+```
+l: List<String> = [Drive, to, develop]
+greeter: (String) -> Unit = (kotlin.String) -> kotlin.Unit
+fun square(Int): Int
+```
diff --git a/docs/interpreter/spark.md b/docs/interpreter/spark.md
index 8c4e118..bd50cb0 100644
--- a/docs/interpreter/spark.md
+++ b/docs/interpreter/spark.md
@@ -26,7 +26,7 @@ limitations under the License.
 ## Overview
 [Apache Spark](http://spark.apache.org) is a fast and general-purpose cluster computing system.
 It provides high-level APIs in Java, Scala, Python and R, and an optimized engine that supports general execution graphs.
-Apache Spark is supported in Zeppelin with Spark interpreter group which consists of below five interpreters.
+Apache Spark is supported in Zeppelin with Spark interpreter group which consists of below six interpreters.
 
 <table class="table-configuration">
   <tr>
@@ -40,6 +40,11 @@ Apache Spark is supported in Zeppelin with Spark interpreter group which consist
     <td>Creates a SparkContext and provides a Scala environment</td>
   </tr>
   <tr>
+    <td>%spark.kotlin</td>
+    <td>KotlinSparkInterpreter</td>
+    <td>Provides a Kotlin environment</td>
+  </tr>
+  <tr>
     <td>%spark.pyspark</td>
     <td>PySparkInterpreter</td>
     <td>Provides a Python environment</td>
@@ -217,7 +222,7 @@ in interpreter setting page means you can use multiple versions of `spark` & `ha
 Starting from 0.9, we totally removed the old spark interpreter implementation, and make the new spark interpreter as the official spark interpreter.
 
 ## SparkContext, SQLContext, SparkSession, ZeppelinContext
-SparkContext, SQLContext and ZeppelinContext are automatically created and exposed as variable names `sc`, `sqlContext` and `z`, respectively, in Scala, Python and R environments.
+SparkContext, SQLContext and ZeppelinContext are automatically created and exposed as variable names `sc`, `sqlContext` and `z`, respectively, in Scala, Kotlin, Python and R environments.
 Staring from 0.6.1 SparkSession is available as variable `spark` when you are using Spark 2.x.
 
 > Note that Scala/Python/R environment shares the same SparkContext, SQLContext and ZeppelinContext instance.
diff --git a/docs/usage/interpreter/installation.md b/docs/usage/interpreter/installation.md
index 83c2094..be718ec 100644
--- a/docs/usage/interpreter/installation.md
+++ b/docs/usage/interpreter/installation.md
@@ -189,6 +189,11 @@ You can also find the below community managed interpreter list in `conf/interpre
     <td>Jdbc interpreter</td>
   </tr>
   <tr>
+    <td>kotlin</td>
+    <td>org.apache.zeppelin:zeppelin-kotlin:0.7.0</td>
+    <td>Kotlin interpreter</td>
+  </tr>
+  <tr>
     <td>kylin</td>
     <td>org.apache.zeppelin:zeppelin-kylin:0.9.0</td>
     <td>Kylin interpreter</td>
diff --git a/kotlin/README.md b/kotlin/README.md
new file mode 100644
index 0000000..0b71c24
--- /dev/null
+++ b/kotlin/README.md
@@ -0,0 +1,83 @@
+# Developer guide to Kotlin interpreter
+
+The following module adds Kotlin language support to Apache Zeppelin.
+Here is the guide to its implementation and how it can be improved and tested. 
+
+## Implementation details
+### Kotlin REPL
+For interactive Kotlin execution, an instance of `KotlinRepl` is created.
+To set REPL properties (such as classpath, generated classes output directory, max result, etc.),
+pass `KotlinReplProperties` to its constructor. For example:
+```$java
+KotlinReplProperties replProperties = new KotlinReplProperties()
+    .maxResult(1000)
+    .shortenTypes(true);
+KotlinRepl repl = new KotlinRepl(replProperties);
+```
+
+### Variable/function binding
+You can also bind variables and functions on REPL creation using implicit receiver language feature.
+This means that all code run in REPL will be executed in Kotlin's `with` block with the receiver, 
+making the receiver's fields and methods accessible.
+   
+To add your variables/functions, extend `KotlinReceiver` class (in separate file), declare your fields and methods, and pass an instance of it to
+`KotlinReplProperties`. Example:
+```$java
+// In separate file:
+class CustomReceiver extends KotlinReceiver {
+    public int myValue = 1 // will be converted to Kotlin "var myValue: Int"
+    public final String messageTemplate = "Value = %VALUE%" // "val messageTemplate: String"
+    
+    public String getMessage() {
+        return messageTemplate.replace("%VALUE%", String.valueOf(myValue));
+    }
+}
+
+// In intepreter creation:
+replProperties.receiver(new CustomReceiver);
+KotlinRepl repl = new KotlinRepl(replProperties);
+repl.eval("getMessage()"); // will return interpreterResult with "Value = 1" string
+``` 
+
+In `KotlinInterpreter` REPL properties are created on construction, are accessible via `getKotlinReplProperties` method,
+and are used in REPL creation on `open()`.
+
+### Generated class files
+Each code snippet run in REPL is registered as a separate class and saved in location 
+specified by `outputDir` REPL property. Anonymous classes and lambdas also get saved there under specific names. 
+
+This is needed for Spark to send classes to remote executors, and in Spark Kotlin interpreter this directory is the same 
+as in `sparkContext` option `spark.repl.class.outputDir`.
+
+### Kotlin Spark Interpreter
+Kotlin interpreter in Spark intepreter group takes `SparkSession`, `JavaSparkContext`, `SQLContext` 
+and `BaseZeppelinContext` from `SparkInterpreter` in the same session and binds them in its scope.
+  
+## Testing
+Kotlin Interpreter and Spark Kotlin Interpreter come with unit tests. 
+They can be run with \
+`mvn clean test` \
+in `$ZEPPELIN_HOME/kotlin` for base Kotlin Interpreter and \
+`mvn -Dtest=KotlinSparkInterpreterTest test` \
+in `$ZEPPELIN_HOME/spark/interpreter` for Spark Kotlin Interpreter.
+
+To test manually, build Zeppelin with \
+`mvn clean package -DskipTests` \
+and create a note with `kotlin` interpreter for base or `spark` for Spark. 
+In Spark interpreter, add `%spark.kotlin` in the start of paragraph to use Kotlin Spark Interpreter.
+
+Example:
+```$kotlin
+%spark.kotlin
+val df = spark.range(10)
+df.show()
+```
+## Possible Improvements
+* It would be great to bind `ZeppelinContext` to base Kotlin interpreter, but for now I had trouble instantiating it 
+inside KotlinInterpreter.
+* When Kotlin has its own Spark API, it will be good to move to it. Currently in Java Spark API Kotlin 
+can not use things like `forEach` because of ambiguity between `Iterable<?>.forEach` and `Map<?, ?>.forEach` 
+(`foreach` from Spark's API does work, though).
+* The scoped mode for Kotlin Spark Interpreter currently has issues with having the same class output directory 
+for different intepreters, leading to overwriting classes. Adding prefixes to generated classes or putting them
+ in separate directories leads to `ClassNotFoundException` on Spark executors.
\ No newline at end of file
diff --git a/kotlin/pom.xml b/kotlin/pom.xml
new file mode 100644
index 0000000..6ab34f6
--- /dev/null
+++ b/kotlin/pom.xml
@@ -0,0 +1,125 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  ~ Licensed to the Apache Software Foundation (ASF) under one or more
+  ~ contributor license agreements. See the NOTICE file distributed with
+  ~ this work for additional information regarding copyright ownership.
+  ~ The ASF licenses this file to You under the Apache License, Version 2.0
+  ~ (the "License"); you may not use this file except in compliance with
+  ~ the License. You may obtain a copy of the License at
+  ~
+  ~ http://www.apache.org/licenses/LICENSE-2.0
+  ~
+  ~ Unless required by applicable law or agreed to in writing, software
+  ~ distributed under the License is distributed on an "AS IS" BASIS,
+  ~ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  ~ See the License for the specific language governing permissions and
+  ~ limitations under the License.
+  -->
+
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <modelVersion>4.0.0</modelVersion>
+
+    <parent>
+        <artifactId>zeppelin-interpreter-parent</artifactId>
+        <groupId>org.apache.zeppelin</groupId>
+        <version>0.9.0-SNAPSHOT</version>
+        <relativePath>../zeppelin-interpreter-parent/pom.xml</relativePath>
+    </parent>
+
+    <groupId>org.apache.zeppelin</groupId>
+    <artifactId>zeppelin-kotlin</artifactId>
+    <packaging>jar</packaging>
+    <version>0.9.0-SNAPSHOT</version>
+    <name>Zeppelin: Kotlin interpreter</name>
+
+    <properties>
+        <interpreter.name>kotlin</interpreter.name>
+        <kotlin.version>1.3.50</kotlin.version>
+    </properties>
+
+    <dependencies>
+        <dependency>
+            <groupId>org.jetbrains.kotlin</groupId>
+            <artifactId>kotlin-scripting-jvm-host-embeddable</artifactId>
+            <version>${kotlin.version}</version>
+            <exclusions>
+                <exclusion>
+                    <groupId>org.jetbrains.kotlin</groupId>
+                    <artifactId>kotlin-stdlib</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>org.jetbrains.kotlin</groupId>
+                    <artifactId>kotlin-stdlib-common</artifactId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+        <dependency>
+            <groupId>org.jetbrains.kotlin</groupId>
+            <artifactId>kotlin-compiler-embeddable</artifactId>
+            <version>${kotlin.version}</version>
+            <exclusions>
+                <exclusion>
+                    <groupId>org.jetbrains.kotlin</groupId>
+                    <artifactId>kotlin-stdlib</artifactId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+        <dependency>
+            <groupId>org.jetbrains.kotlin</groupId>
+            <artifactId>kotlin-scripting-jvm</artifactId>
+            <version>${kotlin.version}</version>
+            <exclusions>
+                <exclusion>
+                    <groupId>org.jetbrains.kotlin</groupId>
+                    <artifactId>kotlin-stdlib</artifactId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+        <dependency>
+            <groupId>org.jetbrains.kotlin</groupId>
+            <artifactId>kotlin-scripting-compiler-embeddable</artifactId>
+            <version>${kotlin.version}</version>
+            <exclusions>
+                <exclusion>
+                    <groupId>org.jetbrains.kotlin</groupId>
+                    <artifactId>kotlin-stdlib</artifactId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+        <dependency>
+            <groupId>org.jetbrains.kotlin</groupId>
+            <artifactId>kotlin-stdlib</artifactId>
+            <version>${kotlin.version}</version>
+        </dependency>
+
+        <dependency>
+            <groupId>org.jetbrains.kotlin</groupId>
+            <artifactId>kotlin-reflect</artifactId>
+            <version>${kotlin.version}</version>
+        </dependency>
+    </dependencies>
+
+    <build>
+        <plugins>
+            <plugin>
+                <artifactId>maven-enforcer-plugin</artifactId>
+            </plugin>
+            <plugin>
+                <artifactId>maven-dependency-plugin</artifactId>
+            </plugin>
+            <plugin>
+                <artifactId>maven-resources-plugin</artifactId>
+            </plugin>
+            <plugin>
+                <artifactId>maven-shade-plugin</artifactId>
+            </plugin>
+            <plugin>
+                <groupId>org.apache.maven.plugins</groupId>
+                <artifactId>maven-checkstyle-plugin</artifactId>
+                <configuration>
+                    <skip>false</skip>
+                </configuration>
+            </plugin>
+        </plugins>
+    </build>
+</project>
diff --git a/kotlin/src/main/java/org/apache/zeppelin/kotlin/KotlinInterpreter.java b/kotlin/src/main/java/org/apache/zeppelin/kotlin/KotlinInterpreter.java
new file mode 100644
index 0000000..8276f5f
--- /dev/null
+++ b/kotlin/src/main/java/org/apache/zeppelin/kotlin/KotlinInterpreter.java
@@ -0,0 +1,189 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES 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.zeppelin.kotlin;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import java.io.File;
+import java.io.PrintStream;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import org.apache.zeppelin.interpreter.Interpreter;
+import org.apache.zeppelin.interpreter.InterpreterContext;
+import org.apache.zeppelin.interpreter.InterpreterException;
+import org.apache.zeppelin.interpreter.InterpreterOutput;
+import org.apache.zeppelin.interpreter.InterpreterResult;
+import org.apache.zeppelin.interpreter.thrift.InterpreterCompletion;
+import org.apache.zeppelin.interpreter.util.InterpreterOutputStream;
+import org.apache.zeppelin.kotlin.completion.KotlinCompleter;
+import org.apache.zeppelin.kotlin.context.KotlinReceiver;
+import org.apache.zeppelin.kotlin.reflect.KotlinFunctionInfo;
+import org.apache.zeppelin.kotlin.reflect.KotlinVariableInfo;
+import org.apache.zeppelin.kotlin.repl.KotlinRepl;
+import org.apache.zeppelin.kotlin.repl.building.KotlinReplProperties;
+import org.apache.zeppelin.scheduler.Job;
+
+public class KotlinInterpreter extends Interpreter {
+
+  private static Logger logger = LoggerFactory.getLogger(KotlinInterpreter.class);
+
+  private InterpreterOutputStream out;
+  private KotlinRepl interpreter;
+  private KotlinReplProperties replProperties;
+  private KotlinCompleter completer;
+
+  public KotlinInterpreter(Properties properties) {
+    super(properties);
+    replProperties = new KotlinReplProperties();
+
+    int maxResult = Integer.parseInt(
+        properties.getProperty("zeppelin.kotlin.maxResult", "1000"));
+
+    boolean shortenTypes = Boolean.parseBoolean(
+        properties.getProperty("zeppelin.kotlin.shortenTypes", "true"));
+    String imports = properties.getProperty("zeppelin.interpreter.localRepo", "");
+
+    completer = new KotlinCompleter();
+    replProperties
+        .receiver(new KotlinReceiver())
+        .maxResult(maxResult)
+        .codeOnLoad("")
+        .classPath(getImportClasspath(imports))
+        .shortenTypes(shortenTypes);
+  }
+
+  public KotlinReplProperties getKotlinReplProperties() {
+    return replProperties;
+  }
+
+  @Override
+  public void open() throws InterpreterException {
+    interpreter = new KotlinRepl(replProperties);
+
+    completer.setCtx(interpreter.getKotlinContext());
+    out = new InterpreterOutputStream(logger);
+  }
+
+  @Override
+  public void close() {
+
+  }
+
+  @Override
+  public InterpreterResult interpret(String code,
+                                     InterpreterContext context) throws InterpreterException{
+    // saving job's running thread for cancelling
+    Job<?> runningJob = getRunningJob(context.getParagraphId());
+    if (runningJob != null) {
+      runningJob.info().put("CURRENT_THREAD", Thread.currentThread());
+    }
+
+    return runWithOutput(code, context.out);
+  }
+
+  @Override
+  public void cancel(InterpreterContext context) throws InterpreterException {
+    Job<?> runningJob = getRunningJob(context.getParagraphId());
+    if (runningJob != null) {
+      Map<String, Object> info = runningJob.info();
+      Object object = info.get("CURRENT_THREAD");
+      if (object instanceof Thread) {
+        try {
+          Thread t = (Thread) object;
+          t.interrupt();
+        } catch (Throwable t) {
+          logger.error("Failed to cancel script: " + t, t);
+        }
+      }
+    }
+  }
+
+  @Override
+  public FormType getFormType() throws InterpreterException {
+    return FormType.NATIVE;
+  }
+
+  @Override
+  public int getProgress(InterpreterContext context) throws InterpreterException {
+    return 0;
+  }
+
+  @Override
+  public List<InterpreterCompletion> completion(String buf, int cursor,
+      InterpreterContext interpreterContext) throws InterpreterException {
+    return completer.completion(buf, cursor, interpreterContext);
+  }
+
+  public List<KotlinVariableInfo> getVariables() {
+    return interpreter.getVariables();
+  }
+
+  public List<KotlinFunctionInfo> getFunctions() {
+    return interpreter.getFunctions();
+  }
+
+  private Job<?> getRunningJob(String paragraphId) {
+    Job foundJob = null;
+    Collection<Job> jobsRunning = getScheduler().getAllJobs();
+    for (Job job : jobsRunning) {
+      if (job.getId().equals(paragraphId)) {
+        foundJob = job;
+      }
+    }
+    return foundJob;
+  }
+
+  /**
+   * Kotlin interpreter uses System.out for printing, so it is redirected to InterpreterOutput.
+   * Note that Scala's Console class needs separate output redirection
+   */
+  private InterpreterResult runWithOutput(String code, InterpreterOutput out) {
+    this.out.setInterpreterOutput(out);
+
+    PrintStream oldOut = System.out;
+    PrintStream newOut = (out != null) ? new PrintStream(out) : null;
+    try {
+      System.setOut(newOut);
+      return interpreter.eval(code);
+    } finally {
+      System.setOut(oldOut);
+    }
+  }
+
+  private List<String> getImportClasspath(String localRepo) {
+    List<String> classpath = new ArrayList<>();
+    if (localRepo.equals("")) {
+      return classpath;
+    }
+
+    File repo = new File(localRepo);
+    File[] files = repo.listFiles();
+    if (files == null) {
+      return classpath;
+    }
+    for (File file : files) {
+      if (file.isFile()) {
+        classpath.add(file.getAbsolutePath());
+      }
+    }
+    return classpath;
+  }
+}
diff --git a/kotlin/src/main/java/org/apache/zeppelin/kotlin/completion/KotlinCompleter.java b/kotlin/src/main/java/org/apache/zeppelin/kotlin/completion/KotlinCompleter.java
new file mode 100644
index 0000000..4b3789f
--- /dev/null
+++ b/kotlin/src/main/java/org/apache/zeppelin/kotlin/completion/KotlinCompleter.java
@@ -0,0 +1,68 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES 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.zeppelin.kotlin.completion;
+
+import static org.apache.zeppelin.kotlin.reflect.KotlinReflectUtil.shorten;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.stream.Collectors;
+import org.apache.zeppelin.interpreter.InterpreterContext;
+import org.apache.zeppelin.interpreter.thrift.InterpreterCompletion;
+import org.apache.zeppelin.kotlin.reflect.KotlinFunctionInfo;
+import org.apache.zeppelin.kotlin.reflect.KotlinVariableInfo;
+import org.apache.zeppelin.kotlin.repl.KotlinRepl;
+
+public class KotlinCompleter {
+  private static final List<InterpreterCompletion> keywords = KotlinKeywords.KEYWORDS.stream()
+      .map(keyword -> new InterpreterCompletion(keyword, keyword, null))
+      .collect(Collectors.toList());
+
+  private KotlinRepl.KotlinContext ctx;
+
+  public void setCtx(KotlinRepl.KotlinContext ctx) {
+    this.ctx = ctx;
+  }
+
+  public List<InterpreterCompletion> completion(String buf, int cursor,
+                                                InterpreterContext interpreterContext)  {
+    if (ctx == null) {
+      return new ArrayList<>(keywords);
+    }
+
+    List<InterpreterCompletion> result = new ArrayList<>();
+
+    for (KotlinVariableInfo var : ctx.getVars()) {
+      result.add(new InterpreterCompletion(
+          var.getName(),
+          var.getName(),
+          shorten(var.getType())
+      ));
+    }
+
+    for (KotlinFunctionInfo fun : ctx.getFunctions()) {
+      result.add(new InterpreterCompletion(
+          fun.getName(),
+          fun.getName(),
+          fun.toString(true)
+      ));
+    }
+
+    result.addAll(keywords);
+    return result;
+  }
+}
diff --git a/kotlin/src/main/java/org/apache/zeppelin/kotlin/completion/KotlinKeywords.java b/kotlin/src/main/java/org/apache/zeppelin/kotlin/completion/KotlinKeywords.java
new file mode 100644
index 0000000..2c75d8c
--- /dev/null
+++ b/kotlin/src/main/java/org/apache/zeppelin/kotlin/completion/KotlinKeywords.java
@@ -0,0 +1,104 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES 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.zeppelin.kotlin.completion;
+
+import java.util.Arrays;
+import java.util.List;
+
+public class KotlinKeywords {
+  /**
+   * List of Kotlin keywords for completion.
+   */
+  public static final List<String> KEYWORDS = Arrays.asList(
+      "as",
+      "as?",
+      "break",
+      "class",
+      "continue",
+      "do",
+      "else",
+      "false",
+      "for",
+      "fun",
+      "if",
+      "in",
+      "interface",
+      "is",
+      "null",
+      "object",
+      "package",
+      "return",
+      "super",
+      "this",
+      "throw",
+      "true",
+      "try",
+      "typealias",
+      "typeof",
+      "val",
+      "var",
+      "when",
+      "while",
+      "by",
+      "catch",
+      "constructor",
+      "delegate",
+      "dynamic",
+      "field",
+      "file",
+      "finally",
+      "get",
+      "import",
+      "init",
+      "param",
+      "property",
+      "receiver",
+      "set",
+      "setparam",
+      "where",
+      "actual",
+      "abstract",
+      "annotation",
+      "companion",
+      "const",
+      "crossinline",
+      "data",
+      "enum",
+      "expect",
+      "external",
+      "final",
+      "infix",
+      "inline",
+      "inner",
+      "internal",
+      "lateinit",
+      "noinline",
+      "open",
+      "operator",
+      "out",
+      "override",
+      "private",
+      "protected",
+      "public",
+      "reified",
+      "sealed",
+      "suspend",
+      "tailrec",
+      "vararg"
+  );
+}
diff --git a/kotlin/src/main/java/org/apache/zeppelin/kotlin/context/KotlinReceiver.java b/kotlin/src/main/java/org/apache/zeppelin/kotlin/context/KotlinReceiver.java
new file mode 100644
index 0000000..2b5ac8f
--- /dev/null
+++ b/kotlin/src/main/java/org/apache/zeppelin/kotlin/context/KotlinReceiver.java
@@ -0,0 +1,37 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES 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.zeppelin.kotlin.context;
+
+import org.apache.zeppelin.kotlin.repl.KotlinRepl;
+
+/**
+ * The implicit receiver for lines in Kotlin REPL.
+ *  It is passed to the script as an implicit receiver, identical to:
+ *  with (context) {
+ *     ...
+ *  }
+ *
+ *  KotlinReceiver can be inherited from and passed to REPL building properties,
+ *  so other variables and functions can be accessed inside REPL.
+ *  By default, it only has KotlinContext.
+ *  Inherited KotlinReceivers should be in separate java file, they can't be inner or nested.
+ */
+public class KotlinReceiver {
+  public KotlinRepl.KotlinContext kc;
+}
+
diff --git a/kotlin/src/main/java/org/apache/zeppelin/kotlin/reflect/ContextUpdater.java b/kotlin/src/main/java/org/apache/zeppelin/kotlin/reflect/ContextUpdater.java
new file mode 100644
index 0000000..33c81d6
--- /dev/null
+++ b/kotlin/src/main/java/org/apache/zeppelin/kotlin/reflect/ContextUpdater.java
@@ -0,0 +1,160 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES 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.zeppelin.kotlin.reflect;
+
+import org.jetbrains.kotlin.cli.common.repl.AggregatedReplStageState;
+import org.jetbrains.kotlin.cli.common.repl.ReplHistoryRecord;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import java.lang.reflect.Field;
+import java.lang.reflect.Method;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+import kotlin.Pair;
+import kotlin.reflect.KFunction;
+import kotlin.reflect.KProperty;
+import kotlin.reflect.jvm.ReflectJvmMapping;
+
+/**
+ * ContextUpdater updates current user-defined functions and variables
+ * to use in completion and KotlinContext.
+ */
+public class ContextUpdater {
+  private static final Logger logger = LoggerFactory.getLogger(ContextUpdater.class);
+  private static final Set<Method> objectMethods =
+      new HashSet<>(Arrays.asList(Object.class.getMethods()));
+  
+  private AggregatedReplStageState<?, ?> state;
+  private Map<String, KotlinVariableInfo> vars;
+  private Set<KotlinFunctionInfo> functions;
+
+  public ContextUpdater(AggregatedReplStageState<?, ?> state,
+                        Map<String, KotlinVariableInfo> vars, 
+                        Set<KotlinFunctionInfo> functions) {
+    this.state = state;
+    this.vars = vars;
+    this.functions = functions;
+  }
+
+  public void update() {
+    try {
+      List<Object> lines = getLines();
+      refreshVariables(lines);
+      refreshMethods(lines);
+    } catch (ReflectiveOperationException | NullPointerException e) {
+      logger.error("Exception updating current variables", e);
+    }
+  }
+
+  private void refreshMethods(List<Object> lines) {
+    functions.clear();
+    for (Object line : lines) {
+      Method[] methods = line.getClass().getMethods();
+      for (Method method : methods) {
+        if (objectMethods.contains(method) || method.getName().equals("main")) {
+          continue;
+        }
+        KFunction<?> function = ReflectJvmMapping.getKotlinFunction(method);
+        if (function == null) {
+          continue;
+        }
+        functions.add(new KotlinFunctionInfo(function));
+      }
+    }
+  }
+
+  private List<Object> getLines() {
+    List<Object> lines = state.getHistory().stream()
+        .map(this::getLineFromRecord)
+        .collect(Collectors.toList());
+
+    Collections.reverse(lines);
+    return lines;
+  }
+
+  /*
+    ReplHistoryRecord class holds a pair of line ID and a compiled Line_N class.
+    This Line class is generated by Kotlin REPL compiler, its base class is Object
+    and it contains the functions and variables declared in the paragraph as its
+    methods and fields. For example, line "val i = 1" can be compiled to Line_0.class
+    with field "public final int i" set to 1.
+    Note that when a new value is reassigned to variable in a separate paragraph,
+    REPL compiler does not change the value in a previously created field,
+    instead it creates a field with the same name in the new Line_(N+1) class.
+  */
+  private Object getLineFromRecord(ReplHistoryRecord<? extends Pair<?, ?>> record) {
+    Object statePair = record.getItem().getSecond();
+    return ((Pair<?, ?>) statePair).getSecond();
+  }
+
+  private Object getImplicitReceiver(Object script)
+      throws ReflectiveOperationException {
+    Field receiverField = script.getClass().getDeclaredField("$$implicitReceiver0");
+    return receiverField.get(script);
+  }
+
+  private void refreshVariables(List<Object> lines) throws ReflectiveOperationException {
+    vars.clear();
+    if (!lines.isEmpty()) {
+      Object receiver = getImplicitReceiver(lines.get(0));
+      findReceiverVariables(receiver);
+    }
+    for (Object line : lines) {
+      findLineVariables(line);
+    }
+  }
+
+  // For lines, we only want fields from top level class
+  private void findLineVariables(Object line) throws IllegalAccessException {
+    Field[] fields = line.getClass().getDeclaredFields();
+    findVariables(fields, line);
+  }
+
+  // For implicit receiver, we want to also get fields in parent classes
+  private void findReceiverVariables(Object receiver) throws IllegalAccessException {
+    List<Field> fieldsList = new ArrayList<>();
+    for (Class<?> cl = receiver.getClass(); cl != null; cl = cl.getSuperclass()) {
+      fieldsList.addAll(Arrays.asList(cl.getDeclaredFields()));
+    }
+    findVariables(fieldsList.toArray(new Field[0]), receiver);
+  }
+
+  private void findVariables(Field[] fields, Object o) throws IllegalAccessException {
+    for (Field field : fields) {
+      String fieldName = field.getName();
+      if (fieldName.contains("$$implicitReceiver")) {
+        continue;
+      }
+
+      field.setAccessible(true);
+      Object value = field.get(o);
+      if (!fieldName.contains("script$")) {
+        KProperty<?> descriptor = ReflectJvmMapping.getKotlinProperty(field);
+        if (descriptor != null) {
+          vars.putIfAbsent(fieldName, new KotlinVariableInfo(value, descriptor));
+        }
+      }
+    }
+  }
+}
diff --git a/kotlin/src/main/java/org/apache/zeppelin/kotlin/reflect/KotlinFunctionInfo.java b/kotlin/src/main/java/org/apache/zeppelin/kotlin/reflect/KotlinFunctionInfo.java
new file mode 100644
index 0000000..c4c4579
--- /dev/null
+++ b/kotlin/src/main/java/org/apache/zeppelin/kotlin/reflect/KotlinFunctionInfo.java
@@ -0,0 +1,69 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES 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.zeppelin.kotlin.reflect;
+
+import static org.apache.zeppelin.kotlin.reflect.KotlinReflectUtil.functionSignature;
+import static org.apache.zeppelin.kotlin.reflect.KotlinReflectUtil.shorten;
+import org.jetbrains.annotations.NotNull;
+import kotlin.reflect.KFunction;
+
+public class KotlinFunctionInfo implements Comparable<KotlinFunctionInfo> {
+  private final KFunction<?> function;
+
+  public KotlinFunctionInfo(KFunction<?> function) {
+    this.function = function;
+  }
+
+  public KFunction<?> getFunction() {
+    return function;
+  }
+
+  public String getName() {
+    return function.getName();
+  }
+
+  public String toString(boolean shortenTypes) {
+    if (shortenTypes) {
+      return shorten(toString());
+    }
+    return toString();
+  }
+
+  @Override
+  public String toString() {
+    return functionSignature(function);
+  }
+
+  @Override
+  public int compareTo(@NotNull KotlinFunctionInfo f) {
+    return this.toString().compareTo(f.toString());
+  }
+
+  @Override
+  public int hashCode() {
+    return this.toString().hashCode();
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (obj instanceof KotlinFunctionInfo) {
+      return this.toString().equals(obj.toString());
+    }
+    return false;
+  }
+}
diff --git a/kotlin/src/main/java/org/apache/zeppelin/kotlin/reflect/KotlinReflectUtil.java b/kotlin/src/main/java/org/apache/zeppelin/kotlin/reflect/KotlinReflectUtil.java
new file mode 100644
index 0000000..9a3d283
--- /dev/null
+++ b/kotlin/src/main/java/org/apache/zeppelin/kotlin/reflect/KotlinReflectUtil.java
@@ -0,0 +1,37 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES 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.zeppelin.kotlin.reflect;
+
+import kotlin.reflect.KFunction;
+
+/**
+ * Util class for pretty-printing Kotlin variables and functions.
+ */
+public class KotlinReflectUtil {
+  public static String functionSignature(KFunction<?> function) {
+    return function.toString().replaceAll("Line_\\d+\\.", "");
+  }
+
+  public static String shorten(String name) {
+    if (name == null) {
+      return null;
+    }
+    // kotlin.collections.List<kotlin.Int> -> List<Int>
+    return name.replaceAll("(\\b[_a-zA-Z$][_a-zA-Z0-9$]*\\b\\.)+", "");
+  }
+}
diff --git a/kotlin/src/main/java/org/apache/zeppelin/kotlin/reflect/KotlinVariableInfo.java b/kotlin/src/main/java/org/apache/zeppelin/kotlin/reflect/KotlinVariableInfo.java
new file mode 100644
index 0000000..a90dacc
--- /dev/null
+++ b/kotlin/src/main/java/org/apache/zeppelin/kotlin/reflect/KotlinVariableInfo.java
@@ -0,0 +1,60 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES 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.zeppelin.kotlin.reflect;
+
+import static org.apache.zeppelin.kotlin.reflect.KotlinReflectUtil.shorten;
+import kotlin.reflect.KProperty;
+
+public class KotlinVariableInfo {
+  private final Object value;
+  private final KProperty<?> descriptor;
+
+  public KotlinVariableInfo(Object value, KProperty<?> descriptor) {
+    this.value = value;
+    this.descriptor = descriptor;
+  }
+
+  public Object getValue() {
+    return value;
+  }
+
+  public KProperty<?> getDescriptor() {
+    return descriptor;
+  }
+
+  public String getName() {
+    return descriptor.getName();
+  }
+
+  public String getType() {
+    return descriptor.getReturnType().toString();
+  }
+
+  public String toString(boolean shortenTypes) {
+    String type = getType();
+    if (shortenTypes) {
+      type = shorten(type);
+    }
+    return getName() + ": " + type + " = " + getValue();
+  }
+
+  @Override
+  public String toString() {
+    return toString(false);
+  }
+}
diff --git a/kotlin/src/main/java/org/apache/zeppelin/kotlin/repl/ClassWriter.java b/kotlin/src/main/java/org/apache/zeppelin/kotlin/repl/ClassWriter.java
new file mode 100644
index 0000000..e7de34c
--- /dev/null
+++ b/kotlin/src/main/java/org/apache/zeppelin/kotlin/repl/ClassWriter.java
@@ -0,0 +1,91 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.zeppelin.kotlin.repl;
+
+import org.jetbrains.kotlin.cli.common.repl.CompiledClassData;
+import org.jetbrains.kotlin.cli.common.repl.ReplCompileResult;
+import org.jetbrains.kotlin.scripting.compiler.plugin.impl.KJvmCompiledModuleInMemory;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import java.io.BufferedOutputStream;
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.util.Objects;
+import kotlin.script.experimental.jvm.impl.KJvmCompiledScript;
+
+/**
+ *  Kotlin REPL compiler generates compiled classes consisting of
+ * compiled in-memory module and some other classes.
+ *  Spark may need saving them somewhere to send them to the executors,
+ * so this class provides writing classes on disk.
+ */
+public class ClassWriter {
+  private static Logger logger = LoggerFactory.getLogger(ClassWriter.class);
+
+  private String outputDir;
+
+  public ClassWriter(String outputDir) {
+    this.outputDir = outputDir;
+  }
+
+  public void writeClasses(ReplCompileResult.CompiledClasses classes) {
+    if (outputDir == null) {
+      return;
+    }
+
+    for (CompiledClassData compiledClass: classes.getClasses()) {
+      String filePath = compiledClass.getPath();
+      if (!filePath.contains(File.separator)) {
+        String classWritePath = outputDir + File.separator + filePath;
+        writeClass(compiledClass.getBytes(), classWritePath);
+      }
+    }
+
+    writeModuleInMemory(classes);
+  }
+
+  private void writeModuleInMemory(ReplCompileResult.CompiledClasses classes) {
+    try {
+      KJvmCompiledScript<?> compiledScript = Objects.requireNonNull(
+          (KJvmCompiledScript<?>) classes.getData());
+
+      KJvmCompiledModuleInMemory moduleInMemory = Objects.requireNonNull(
+          (KJvmCompiledModuleInMemory) compiledScript.getCompiledModule());
+
+      moduleInMemory.getCompilerOutputFiles().forEach((name, bytes) -> {
+        if (name.contains("class")) {
+          writeClass(bytes, outputDir + File.separator + name);
+        }
+      });
+    } catch (ClassCastException | NullPointerException e) {
+      logger.info("Compiled line #" + classes.getLineId().getNo() + "has no in-memory modules");
+    }
+  }
+
+  private void writeClass(byte[] classBytes, String path) {
+    try (FileOutputStream fos = new FileOutputStream(path);
+         OutputStream out = new BufferedOutputStream(fos)) {
+      out.write(classBytes);
+      out.flush();
+    } catch (IOException e) {
+      logger.error(e.getMessage());
+    }
+  }
+}
diff --git a/kotlin/src/main/java/org/apache/zeppelin/kotlin/repl/KotlinRepl.java b/kotlin/src/main/java/org/apache/zeppelin/kotlin/repl/KotlinRepl.java
new file mode 100644
index 0000000..38fdba3
--- /dev/null
+++ b/kotlin/src/main/java/org/apache/zeppelin/kotlin/repl/KotlinRepl.java
@@ -0,0 +1,272 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES 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.zeppelin.kotlin.repl;
+
+import static org.apache.zeppelin.kotlin.reflect.KotlinReflectUtil.shorten;
+import org.jetbrains.kotlin.cli.common.repl.AggregatedReplStageState;
+import org.jetbrains.kotlin.cli.common.repl.InvokeWrapper;
+import org.jetbrains.kotlin.cli.common.repl.ReplCodeLine;
+import org.jetbrains.kotlin.cli.common.repl.ReplCompileResult;
+import org.jetbrains.kotlin.cli.common.repl.ReplEvalResult;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.stream.Collectors;
+import kotlin.jvm.functions.Function0;
+import kotlin.script.experimental.jvmhost.repl.JvmReplCompiler;
+import kotlin.script.experimental.jvmhost.repl.JvmReplEvaluator;
+import org.apache.zeppelin.interpreter.InterpreterResult;
+import org.apache.zeppelin.kotlin.reflect.ContextUpdater;
+import org.apache.zeppelin.kotlin.reflect.KotlinFunctionInfo;
+import org.apache.zeppelin.kotlin.reflect.KotlinVariableInfo;
+import org.apache.zeppelin.kotlin.repl.building.KotlinReplProperties;
+import org.apache.zeppelin.kotlin.repl.building.ReplBuilding;
+
+/**
+ * Read-evaluate-print loop for Kotlin code.
+ * Each code snippet is compiled into Line_N class and evaluated.
+ *
+ * Outside variables and functions can be bound to REPL
+ * by inheriting KotlinReceiver class and passing it to REPL properties on creation.
+ * After that, all fields and methods of receiver are seen inside the snippet scope
+ * as if the code was run in Kotlin's `with` block.
+ *
+ * By default, KotlinReceiver has KotlinContext bound by the name `kc`.
+ * It can be used to show user-defined variables and functions
+ * and setting invokeWrapper to add effects to snippet evaluation.
+ */
+public class KotlinRepl {
+  private static Logger logger = LoggerFactory.getLogger(KotlinRepl.class);
+
+  private JvmReplCompiler compiler;
+  private JvmReplEvaluator evaluator;
+  private AggregatedReplStageState<?, ?> state;
+  private AtomicInteger counter;
+  private ClassWriter writer;
+  private KotlinContext ctx;
+  private InvokeWrapper wrapper;
+  private int maxResult;
+  private ContextUpdater contextUpdater;
+  boolean shortenTypes;
+
+  private KotlinRepl() { }
+
+  @SuppressWarnings("unchecked")
+  public KotlinRepl(KotlinReplProperties properties) {
+    compiler = ReplBuilding.buildCompiler(properties);
+    evaluator = ReplBuilding.buildEvaluator(properties);
+    ReentrantReadWriteLock stateLock = new ReentrantReadWriteLock();
+    state = new AggregatedReplStageState(
+        compiler.createState(stateLock),
+        evaluator.createState(stateLock),
+        stateLock);
+    counter = new AtomicInteger(0);
+
+    writer = new ClassWriter(properties.getOutputDir());
+    maxResult = properties.getMaxResult();
+    shortenTypes = properties.getShortenTypes();
+
+    ctx = new KotlinContext();
+    properties.getReceiver().kc = ctx;
+
+    contextUpdater = new ContextUpdater(
+        state, ctx.vars, ctx.functions);
+
+    for (String line: properties.getCodeOnLoad()) {
+      eval(line);
+    }
+  }
+
+  public List<KotlinVariableInfo> getVariables() {
+    return ctx.getVars();
+  }
+
+  public List<KotlinFunctionInfo> getFunctions() {
+    return ctx.getFunctions();
+  }
+
+  public KotlinContext getKotlinContext() {
+    return ctx;
+  }
+
+  /**
+   * Evaluates code snippet and returns interpreter result.
+   * REPL evaluation consists of:
+   * - Compiling code in JvmReplCompiler
+   * - Writing compiled classes to disk
+   * - Evaluating compiled classes inside InvokeWrapper
+   * - Updating list of user-defined functions and variables
+   * - Formatting result
+   * @param code Kotlin code to execute
+   * @return result of interpretation
+   */
+  public InterpreterResult eval(String code) {
+    ReplCompileResult compileResult = compiler.compile(state,
+        new ReplCodeLine(counter.getAndIncrement(), 0, code));
+
+    Optional<InterpreterResult> compileError = checkCompileError(compileResult);
+    if (compileError.isPresent()) {
+      return compileError.get();
+    }
+
+    ReplCompileResult.CompiledClasses classes =
+        (ReplCompileResult.CompiledClasses) compileResult;
+    writer.writeClasses(classes);
+
+    ReplEvalResult evalResult  = evalInWrapper(classes);
+
+    Optional<InterpreterResult> evalError = checkEvalError(evalResult);
+    if (evalError.isPresent()) {
+      return evalError.get();
+    }
+
+    contextUpdater.update();
+
+    if (evalResult instanceof ReplEvalResult.UnitResult) {
+      return new InterpreterResult(InterpreterResult.Code.SUCCESS);
+    }
+    if (evalResult instanceof ReplEvalResult.ValueResult) {
+      ReplEvalResult.ValueResult v = (ReplEvalResult.ValueResult) evalResult;
+      String typeString = shortenTypes ? shorten(v.getType()) : v.getType();
+      String valueString = prepareValueString(v.getValue());
+
+      return new InterpreterResult(
+          InterpreterResult.Code.SUCCESS,
+          v.getName() + ": " + typeString + " = " + valueString);
+    }
+    return new InterpreterResult(InterpreterResult.Code.ERROR,
+        "unknown evaluation result: " + evalResult.toString());
+  }
+
+  private ReplEvalResult evalInWrapper(ReplCompileResult.CompiledClasses classes) {
+    ReplEvalResult evalResult;
+    // For now, invokeWrapper parameter in evaluator.eval does not work, so wrapping happens here
+    Function0<ReplEvalResult> runEvaluator = () -> evaluator.eval(state, classes, null, null);
+    if (wrapper != null) {
+      evalResult = wrapper.invoke(runEvaluator);
+    } else {
+      evalResult = runEvaluator.invoke();
+    }
+    return evalResult;
+  }
+
+  private Optional<InterpreterResult> checkCompileError(ReplCompileResult compileResult) {
+    if (compileResult instanceof ReplCompileResult.Incomplete) {
+      return Optional.of(new InterpreterResult(InterpreterResult.Code.INCOMPLETE));
+    }
+
+    if (compileResult instanceof ReplCompileResult.Error) {
+      ReplCompileResult.Error e = (ReplCompileResult.Error) compileResult;
+      return Optional.of(new InterpreterResult(InterpreterResult.Code.ERROR, e.getMessage()));
+    }
+
+    if (!(compileResult instanceof ReplCompileResult.CompiledClasses)) {
+      return Optional.of(new InterpreterResult(InterpreterResult.Code.ERROR,
+          "unknown compilation result:" + compileResult.toString()));
+    }
+
+    return Optional.empty();
+  }
+
+  private Optional<InterpreterResult> checkEvalError(ReplEvalResult evalResult) {
+    if (evalResult instanceof ReplEvalResult.Error) {
+      ReplEvalResult.Error e = (ReplEvalResult.Error) evalResult;
+      return Optional.of(new InterpreterResult(InterpreterResult.Code.ERROR, e.getMessage()));
+    }
+
+    if (evalResult instanceof ReplEvalResult.Incomplete) {
+      return Optional.of(new InterpreterResult(InterpreterResult.Code.INCOMPLETE));
+    }
+
+    if (evalResult instanceof ReplEvalResult.HistoryMismatch) {
+      ReplEvalResult.HistoryMismatch e = (ReplEvalResult.HistoryMismatch) evalResult;
+      return Optional.of(new InterpreterResult(
+          InterpreterResult.Code.ERROR, "history mismatch at " + e.getLineNo()));
+    }
+
+    return Optional.empty();
+  }
+
+  private String prepareValueString(Object value) {
+    if (value == null) {
+      return "null";
+    }
+    if (!(value instanceof Collection<?>)) {
+      return value.toString();
+    }
+
+    Collection<?> collection = (Collection<?>) value;
+
+    if (collection.size() <= maxResult) {
+      return value.toString();
+    }
+
+    return "[" + collection.stream()
+        .limit(maxResult)
+        .map(Object::toString)
+        .collect(Collectors.joining(","))
+        + " ... " + (collection.size() - maxResult) + " more]";
+  }
+
+  /**
+   * Kotlin REPL has built-in context for getting user-declared functions and variables
+   * and setting invokeWrapper for additional side effects in evaluation.
+   * It can be accessed inside REPL by name `kc`, e.g. kc.showVars()
+   */
+  public class KotlinContext {
+    private Map<String, KotlinVariableInfo> vars = new HashMap<>();
+    private Set<KotlinFunctionInfo> functions = new TreeSet<>();
+
+    public List<KotlinVariableInfo> getVars() {
+      return new ArrayList<>(vars.values());
+    }
+
+    public void setWrapper(InvokeWrapper wrapper) {
+      KotlinRepl.this.wrapper = wrapper;
+    }
+
+    public InvokeWrapper getWrapper() {
+      return KotlinRepl.this.wrapper;
+    }
+
+    public List<KotlinFunctionInfo> getFunctions() {
+      return new ArrayList<>(functions);
+    }
+
+    public void showVars() {
+      for (KotlinVariableInfo var : vars.values()) {
+        System.out.println(var.toString(shortenTypes));
+      }
+    }
+
+    public void showFunctions() {
+      for (KotlinFunctionInfo fun : functions) {
+        System.out.println(fun.toString(shortenTypes));
+      }
+    }
+  }
+}
diff --git a/kotlin/src/main/java/org/apache/zeppelin/kotlin/repl/building/KotlinReplProperties.java b/kotlin/src/main/java/org/apache/zeppelin/kotlin/repl/building/KotlinReplProperties.java
new file mode 100644
index 0000000..a3e1b74
--- /dev/null
+++ b/kotlin/src/main/java/org/apache/zeppelin/kotlin/repl/building/KotlinReplProperties.java
@@ -0,0 +1,129 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES 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.zeppelin.kotlin.repl.building;
+
+import static kotlin.script.experimental.jvm.JvmScriptingHostConfigurationKt.getDefaultJvmScriptingHostConfiguration;
+import java.io.File;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import kotlin.script.experimental.host.ScriptingHostConfiguration;
+import org.apache.zeppelin.kotlin.context.KotlinReceiver;
+
+/**
+ * Class that holds properties for Kotlin REPL creation,
+ * namely implicit receiver, classpath, preloaded code, directory for class bytecode output,
+ * max result limit and shortening types flag.
+ *
+ * Set its parameters by chaining corresponding methods, e.g.
+ * properties.outputDir(dir).shortenTypes(false)
+ *
+ * Get its parameters via getters.
+ */
+public class KotlinReplProperties {
+
+  private ScriptingHostConfiguration hostConf = getDefaultJvmScriptingHostConfiguration();
+
+  private KotlinReceiver receiver;
+  private Set<String> classpath;
+  private List<String> codeOnLoad;
+  private String outputDir;
+  private int maxResult = 1000;
+  private boolean shortenTypes = true;
+
+  public KotlinReplProperties() {
+    this.receiver = new KotlinReceiver();
+
+    this.classpath = new HashSet<>();
+    String[] javaClasspath = System.getProperty("java.class.path").split(File.pathSeparator);
+    Collections.addAll(classpath, javaClasspath);
+
+    this.codeOnLoad = new ArrayList<>();
+  }
+
+  public KotlinReplProperties receiver(KotlinReceiver receiver) {
+    this.receiver = receiver;
+    return this;
+  }
+
+  public KotlinReplProperties classPath(String path) {
+    this.classpath.add(path);
+    return this;
+  }
+
+  public KotlinReplProperties classPath(Collection<String> paths) {
+    this.classpath.addAll(paths);
+    return this;
+  }
+
+  public KotlinReplProperties codeOnLoad(String code) {
+    this.codeOnLoad.add(code);
+    return this;
+  }
+
+  public KotlinReplProperties codeOnLoad(Collection<String> code) {
+    this.codeOnLoad.addAll(code);
+    return this;
+  }
+
+  public KotlinReplProperties outputDir(String outputDir) {
+    this.outputDir = outputDir;
+    return this;
+  }
+
+  public KotlinReplProperties maxResult(int maxResult) {
+    this.maxResult = maxResult;
+    return this;
+  }
+
+  public KotlinReplProperties shortenTypes(boolean shortenTypes) {
+    this.shortenTypes = shortenTypes;
+    return this;
+  }
+
+  public ScriptingHostConfiguration getHostConf() {
+    return hostConf;
+  }
+
+  public KotlinReceiver getReceiver() {
+    return receiver;
+  }
+
+  public Set<String> getClasspath() {
+    return classpath;
+  }
+
+  public List<String> getCodeOnLoad() {
+    return codeOnLoad;
+  }
+
+  public String getOutputDir() {
+    return outputDir;
+  }
+
+  public int getMaxResult() {
+    return maxResult;
+  }
+
+  public boolean getShortenTypes() {
+    return shortenTypes;
+  }
+}
diff --git a/kotlin/src/main/java/org/apache/zeppelin/kotlin/repl/building/ReplBuilding.java b/kotlin/src/main/java/org/apache/zeppelin/kotlin/repl/building/ReplBuilding.java
new file mode 100644
index 0000000..af50a7d
--- /dev/null
+++ b/kotlin/src/main/java/org/apache/zeppelin/kotlin/repl/building/ReplBuilding.java
@@ -0,0 +1,108 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES 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.zeppelin.kotlin.repl.building;
+
+import org.jetbrains.kotlin.scripting.compiler.plugin.impl.KJvmReplCompilerImpl;
+import java.io.File;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.StringJoiner;
+import kotlin.Unit;
+import kotlin.script.experimental.api.KotlinType;
+import kotlin.script.experimental.api.ScriptCompilationConfiguration;
+import kotlin.script.experimental.api.ScriptCompilationKt;
+import kotlin.script.experimental.api.ScriptEvaluationConfiguration;
+import kotlin.script.experimental.api.ScriptEvaluationKt;
+import kotlin.script.experimental.jvm.BasicJvmScriptEvaluator;
+import kotlin.script.experimental.jvm.JvmScriptCompilationConfigurationBuilder;
+import kotlin.script.experimental.jvm.JvmScriptCompilationKt;
+import kotlin.script.experimental.jvmhost.impl.JvmHostUtilKt;
+import kotlin.script.experimental.jvmhost.repl.JvmReplCompiler;
+import kotlin.script.experimental.jvmhost.repl.JvmReplEvaluator;
+
+/**
+ * Util class for building REPL components.
+ */
+public class ReplBuilding {
+  public static JvmReplCompiler buildCompiler(KotlinReplProperties properties) {
+    String receiverClassPath = properties.getReceiver().getClass()
+        .getProtectionDomain().getCodeSource().getLocation().getPath();
+    properties.classPath(receiverClassPath);
+
+    KJvmReplCompilerImpl compilerImpl =
+        new KJvmReplCompilerImpl(JvmHostUtilKt.withDefaults(properties.getHostConf()));
+
+    return new JvmReplCompiler(
+        buildCompilationConfiguration(properties),
+        properties.getHostConf(),
+        compilerImpl);
+  }
+
+  public static JvmReplEvaluator buildEvaluator(KotlinReplProperties properties) {
+    return new JvmReplEvaluator(
+        buildEvaluationConfiguration(properties),
+        new BasicJvmScriptEvaluator());
+  }
+
+  private static String buildClassPath(KotlinReplProperties p) {
+    StringJoiner joiner = new StringJoiner(File.pathSeparator);
+    for (String path : p.getClasspath()) {
+      if (path != null && !path.equals("")) {
+        joiner.add(path);
+      }
+    }
+    return joiner.toString();
+  }
+
+  private static ScriptCompilationConfiguration buildCompilationConfiguration(
+      KotlinReplProperties p) {
+    return new ScriptCompilationConfiguration((b) -> {
+      b.invoke(ScriptCompilationKt.getHostConfiguration(b), p.getHostConf());
+
+      JvmScriptCompilationConfigurationBuilder jvmBuilder =
+          JvmScriptCompilationKt.getJvm(b);
+      JvmScriptCompilationKt.dependenciesFromCurrentContext(
+          jvmBuilder, new String[0], true, false);
+
+      List<String> compilerOptions = Arrays.asList("-classpath", buildClassPath(p));
+
+      b.invoke(ScriptCompilationKt.getCompilerOptions(b), compilerOptions);
+
+      KotlinType kt = new KotlinType(p.getReceiver().getClass().getCanonicalName());
+      List<KotlinType> receivers =
+          Collections.singletonList(kt);
+      b.invoke(ScriptCompilationKt.getImplicitReceivers(b), receivers);
+
+      return Unit.INSTANCE;
+    });
+  }
+
+  private static ScriptEvaluationConfiguration buildEvaluationConfiguration(
+      KotlinReplProperties p) {
+    return new ScriptEvaluationConfiguration((b) -> {
+      b.invoke(ScriptEvaluationKt.getHostConfiguration(b), p.getHostConf());
+
+      List<Object> receivers =
+          Collections.singletonList(p.getReceiver());
+      b.invoke(ScriptEvaluationKt.getImplicitReceivers(b), receivers);
+
+      return Unit.INSTANCE;
+    });
+  }
+}
diff --git a/kotlin/src/main/resources/interpreter-setting.json b/kotlin/src/main/resources/interpreter-setting.json
new file mode 100644
index 0000000..1277b59
--- /dev/null
+++ b/kotlin/src/main/resources/interpreter-setting.json
@@ -0,0 +1,28 @@
+[
+  {
+    "group": "kotlin",
+    "name": "kotlin",
+    "className": "org.apache.zeppelin.kotlin.KotlinInterpreter",
+    "defaultInterpreter": true,
+    "properties": {
+      "zeppelin.kotlin.maxResult": {
+        "envName": null,
+        "propertyName": "zeppelin.kotlin.maxResult",
+        "defaultValue": "1000",
+        "description": "Max number of dataframe rows to display.",
+        "type": "number"
+      },
+      "zeppelin.kotlin.shortenTypes": {
+        "envName": null,
+        "propertyName": "zeppelin.kotlin.shortenTypes",
+        "defaultValue": true,
+        "description": "Show short types instead of full, e.g. List<String> or kotlin.collections.List<kotlin.String>",
+        "type": "checkbox"
+      }
+    },
+    "editor": {
+      "language": "kotlin",
+      "editOnDblClick": false
+    }
+  }
+]
diff --git a/kotlin/src/test/java/org/apache/zeppelin/kotlin/KotlinInterpreterTest.java b/kotlin/src/test/java/org/apache/zeppelin/kotlin/KotlinInterpreterTest.java
new file mode 100644
index 0000000..a474b3a
--- /dev/null
+++ b/kotlin/src/test/java/org/apache/zeppelin/kotlin/KotlinInterpreterTest.java
@@ -0,0 +1,326 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES 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.zeppelin.kotlin;
+
+import static org.apache.zeppelin.interpreter.InterpreterResult.Code.ERROR;
+import static org.apache.zeppelin.interpreter.InterpreterResult.Code.SUCCESS;
+import static org.apache.zeppelin.kotlin.reflect.KotlinReflectUtil.shorten;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Properties;
+import java.util.stream.Collectors;
+import org.apache.zeppelin.interpreter.InterpreterContext;
+import org.apache.zeppelin.interpreter.InterpreterException;
+import org.apache.zeppelin.interpreter.InterpreterOutput;
+import org.apache.zeppelin.interpreter.InterpreterOutputListener;
+import org.apache.zeppelin.interpreter.InterpreterResult;
+import org.apache.zeppelin.interpreter.InterpreterResultMessageOutput;
+import org.apache.zeppelin.interpreter.thrift.InterpreterCompletion;
+import org.apache.zeppelin.kotlin.reflect.KotlinFunctionInfo;
+import org.apache.zeppelin.kotlin.reflect.KotlinVariableInfo;
+
+
+public class KotlinInterpreterTest {
+
+  private static KotlinInterpreter interpreter;
+  private static InterpreterContext context;
+
+  private static volatile String output = "";
+
+  public void prepareInterpreter() {
+    prepareInterpreter(new Properties());
+  }
+
+  public void prepareInterpreter(Properties properties) {
+    context = getInterpreterContext();
+    interpreter = new KotlinInterpreter(properties);
+    output = "";
+  }
+
+  @Before
+  public void setUp() throws InterpreterException {
+    prepareInterpreter();
+    interpreter.open();
+  }
+
+  @After
+  public void tearDown() {
+    interpreter.close();
+  }
+
+  private static void testCodeForResult(String code, String expected) throws Exception {
+    InterpreterResult result = interpreter.interpret(code, context);
+
+    String value;
+    if (result.message().isEmpty()) {
+      value = "";
+    } else {
+      String message = result.message().get(0).getData().trim();
+      // "res0 : kotlin.Int = 1" -> "kotlin.Int = 1"
+      value = message.substring(message.indexOf(':') + 2);
+    }
+
+    assertEquals(SUCCESS, result.code());
+    assertEquals(expected, value);
+  }
+
+  @Test
+  public void testLiteral() throws Exception {
+    testCodeForResult("1", "Int = 1");
+  }
+
+  @Test
+  public void testOperation() throws Exception {
+    testCodeForResult("\"foo\" + \"bar\"", "String = foobar");
+  }
+
+  @Test
+  public void testFunction() throws Exception {
+    testCodeForResult(
+        "fun square(x: Int): Int = x * x\nsquare(10)",
+        "Int = 100");
+  }
+
+  @Test
+  public void testIncomplete() throws Exception {
+    InterpreterResult result = interpreter.interpret("val x =", context);
+    assertEquals(ERROR, result.code());
+  }
+
+  @Test
+  public void testCompileError() throws Exception {
+    InterpreterResult result = interpreter.interpret("prinln(1)", context);
+    assertEquals(ERROR, result.code());
+    assertEquals("Unresolved reference: prinln", result.message().get(0).getData().trim());
+  }
+
+  @Test
+  public void testOutput() throws Exception {
+    testCodeForResult("println(\"Hello Kotlin\")", "");
+    assertEquals("Hello Kotlin\n", output);
+  }
+
+  @Test
+  public void testRuntimeError() throws Exception {
+    InterpreterResult result = interpreter.interpret(
+        "throw RuntimeException(\"Error Message\")", context);
+    assertEquals(ERROR, result.code());
+    assertEquals("Error Message", result.message().get(0).getData().trim());
+  }
+
+  @Test
+  public void testCancel() throws Exception {
+    Thread t = new Thread(() -> {
+      try {
+        InterpreterResult result = interpreter.interpret(
+            "repeat(10000000) { Thread.sleep(100) }", context);
+        assertEquals(ERROR, result.code());
+        assertEquals("sleep interrupted", result.message().get(0).getData().trim());
+      } catch (InterpreterException e) {
+        Assert.fail(e.getMessage());
+      }
+    });
+    t.start();
+    Thread.sleep(200);
+    interpreter.cancel(context);
+  }
+
+  @Test
+  public void testVariables() throws Exception {
+    interpreter.interpret("val x = 1", context);
+    interpreter.interpret("val x = 2", context);
+    List<KotlinVariableInfo> vars = interpreter.getVariables();
+    assertEquals(2, vars.size());
+
+    KotlinVariableInfo varX = vars.stream()
+        .filter(info -> info.getName().equals("x"))
+        .findFirst()
+        .orElseGet( () -> {
+          Assert.fail();
+          return null;
+        });
+
+    assertEquals(2, varX.getValue());
+    assertEquals("kotlin.Int", varX.getType());
+  }
+
+  @Test
+  public void testGetVariablesFromCode() throws Exception {
+    interpreter.interpret("val x = 1", context);
+    interpreter.interpret("val y = 2", context);
+    interpreter.interpret("val x = 3", context);
+    interpreter.interpret("val l = listOf(1,2,3)", context);
+    interpreter.interpret("kc.showVars()", context);
+    System.out.println(output);
+    assertTrue(output.contains("x: Int = 3"));
+    assertTrue(output.contains("y: Int = 2"));
+    assertTrue(output.contains("l: List<Int> = [1, 2, 3]"));
+    InterpreterResult res = interpreter.interpret("kc.vars = null", context);
+    assertTrue(res.message().get(0).getData().contains("Val cannot be reassigned"));
+  }
+
+  @Test
+  public void testFunctionsAsValues() throws Exception {
+    System.out.println(interpreter.interpret("val f = { x: Int -> x + 1 }", context));
+    System.out.println(interpreter.getVariables());
+  }
+
+  @Test
+  public void testMethods() throws Exception {
+    interpreter.interpret("fun sq(x: Int): Int = x * x", context);
+    interpreter.interpret("fun sq(x: Int): Int = x * x", context);
+    assertEquals(1, interpreter.getFunctions().size());
+
+    interpreter.interpret("fun <T> singletonListOf(elem: T): List<T> = listOf(elem)", context);
+    List<String> signatures = interpreter.getFunctions().stream()
+        .map(KotlinFunctionInfo::toString).collect(Collectors.toList());
+    System.out.println(signatures);
+    assertTrue(signatures.stream().anyMatch(signature ->
+        signature.equals("fun sq(kotlin.Int): kotlin.Int")));
+    assertTrue(signatures.stream().anyMatch(signature ->
+        signature.equals("fun singletonListOf(T): kotlin.collections.List<T>")));
+  }
+
+  @Test
+  public void testCompletion() throws Exception {
+    interpreter.interpret("val x = 1", context);
+    interpreter.interpret("fun inc(n: Int): Int = n + 1", context);
+    List<InterpreterCompletion> completions = interpreter.completion("", 0, context);
+    assertTrue(completions.stream().anyMatch(c -> c.name.equals("x")));
+    assertTrue(completions.stream().anyMatch(c -> c.name.equals("inc")));
+  }
+
+  @Test
+  public void testOutputClasses() throws Exception {
+    prepareInterpreter();
+    Path tempPath = Files.createTempDirectory("tempKotlinClasses");
+    interpreter.getKotlinReplProperties().outputDir(tempPath.toAbsolutePath().toString());
+    interpreter.open();
+    interpreter.interpret("val x = 1\nx", context);
+    File[] dir = tempPath.toFile().listFiles();
+    assertNotNull(dir);
+    assertTrue(dir.length > 0);
+    System.out.println(tempPath);
+    assertTrue(Arrays.stream(dir)
+        .anyMatch(file -> file.getName().matches("Line_\\d+\\.class")));
+    int oldLength = dir.length;
+    interpreter.interpret("x + 1", context);
+    dir = tempPath.toFile().listFiles();
+    assertNotNull(dir);
+    assertTrue(dir.length > oldLength);
+  }
+
+  @Test
+  public void testWrapper() throws Exception {
+    String code = "import org.jetbrains.kotlin.cli.common.repl.InvokeWrapper\n" +
+            "var k = 0\n" +
+            "val wrapper = object : InvokeWrapper {\n" +
+            "    override operator fun <T> invoke(body: () -> T): T {\n" +
+            "        println(\"START\")\n" +
+            "        val result = body()\n" +
+            "        println(\"END\")\n" +
+            "        k = k + 1\n" +
+            "        return result\n" +
+            "    }\n" +
+            "}\n" +
+            "kc.setWrapper(wrapper)\n";
+    interpreter.interpret(code, context);
+    interpreter.interpret("println(\"hello!\")", context);
+    List<KotlinVariableInfo> vars = interpreter.getVariables();
+    for (KotlinVariableInfo v: vars) {
+      if (v.getName().equals("k")) {
+        assertEquals(1, v.getValue());
+      }
+    }
+
+    InterpreterResult result = interpreter.interpret("kc.vars", context);
+    assertTrue(result.message().get(0).getData().contains("k: kotlin.Int = 1"));
+  }
+
+  @Test
+  public void testReflectUtil() throws Exception {
+    String message = interpreter.interpret("1", context)
+        .message().get(0).getData();
+    assertTrue(shorten(message).contains("Int = 1"));
+
+    interpreter.interpret("val f = { l: List<Int> -> l[0] }", context);
+    message = interpreter.interpret("f", context)
+        .message().get(0).getData();
+    assertTrue(shorten(message).contains("(List<Int>) -> Int"));
+
+    interpreter.interpret("fun first(s: String): Char = s[0]", context);
+    KotlinFunctionInfo first = interpreter.getFunctions().get(0);
+    assertEquals("fun first(String): Char", first.toString(true));
+  }
+
+  @Test
+  public void fullTypeNamesTest() throws Exception {
+    prepareInterpreter();
+    interpreter.getKotlinReplProperties().shortenTypes(false);
+    interpreter.open();
+
+    interpreter.interpret("val s = \"abc\"", context);
+    interpreter.interpret("fun f(l: List<String>) { }", context);
+    interpreter.interpret("kc.showFunctions()", context);
+    assertEquals("fun f(kotlin.collections.List<kotlin.String>): kotlin.Unit\n", output);
+    output = "";
+    interpreter.interpret("kc.showVars()", context);
+    System.out.println(output);
+    assertTrue(output.contains("s: kotlin.String = abc"));
+  }
+
+  private static InterpreterContext getInterpreterContext() {
+    output = "";
+    InterpreterContext context = InterpreterContext.builder()
+        .setInterpreterOut(new InterpreterOutput(null))
+        .build();
+    context.out = new InterpreterOutput(
+        new InterpreterOutputListener() {
+          @Override
+          public void onUpdateAll(InterpreterOutput out) {
+
+          }
+
+          @Override
+          public void onAppend(int index, InterpreterResultMessageOutput out, byte[] line) {
+            try {
+              output = out.toInterpreterResultMessage().getData();
+            } catch (IOException e) {
+              e.printStackTrace();
+            }
+          }
+
+          @Override
+          public void onUpdate(int index, InterpreterResultMessageOutput out) {
+
+          }
+        });
+    return context;
+  }
+}
diff --git a/pom.xml b/pom.xml
index 2ec7755..8493dad 100644
--- a/pom.xml
+++ b/pom.xml
@@ -57,6 +57,7 @@
     <module>zeppelin-interpreter-api</module>
     <module>zeppelin-zengine</module>
     <module>zeppelin-display</module>
+    <module>kotlin</module>
     <module>groovy</module>
     <module>spark</module>
     <module>submarine</module>
diff --git a/scalding/pom.xml b/scalding/pom.xml
index 663ad58..e361810 100644
--- a/scalding/pom.xml
+++ b/scalding/pom.xml
@@ -55,6 +55,13 @@
       <name>Twitter Maven Repo</name>
       <url>https://maven.twttr.com</url>
     </repository>
+
+    <!-- Temporary repo -->
+    <repository>
+      <id>zeppelin-dependencies</id>
+      <name>bintray</name>
+      <url>https://jetbrains.bintray.com/zeppelin-dependencies</url>
+    </repository>
   </repositories>
 
   <dependencies>
diff --git a/spark/interpreter/pom.xml b/spark/interpreter/pom.xml
index ba2fbd2..6e5c67a 100644
--- a/spark/interpreter/pom.xml
+++ b/spark/interpreter/pom.xml
@@ -103,6 +103,12 @@
 
     <dependency>
       <groupId>org.apache.zeppelin</groupId>
+      <artifactId>zeppelin-kotlin</artifactId>
+      <version>${project.version}</version>
+    </dependency>
+
+    <dependency>
+      <groupId>org.apache.zeppelin</groupId>
       <artifactId>zeppelin-python</artifactId>
       <version>${project.version}</version>
       <exclusions>
@@ -346,7 +352,7 @@
         <configuration>
           <forkCount>1</forkCount>
           <reuseForks>false</reuseForks>
-          <argLine>-Xmx2048m -XX:MaxPermSize=256m</argLine>
+          <argLine>-Xmx3072m -XX:MaxPermSize=256m</argLine>
           <excludes>
             <exclude>**/SparkRInterpreterTest.java</exclude>
             <exclude>${pyspark.test.exclude}</exclude>
diff --git a/spark/interpreter/src/main/java/org/apache/zeppelin/spark/KotlinSparkInterpreter.java b/spark/interpreter/src/main/java/org/apache/zeppelin/spark/KotlinSparkInterpreter.java
new file mode 100644
index 0000000..4592180
--- /dev/null
+++ b/spark/interpreter/src/main/java/org/apache/zeppelin/spark/KotlinSparkInterpreter.java
@@ -0,0 +1,198 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES 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.zeppelin.spark;
+
+import static org.apache.zeppelin.spark.Utils.buildJobDesc;
+import static org.apache.zeppelin.spark.Utils.buildJobGroupId;
+import org.apache.spark.SparkConf;
+import org.apache.spark.api.java.JavaSparkContext;
+import org.apache.spark.util.Utils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import java.io.File;
+import java.io.IOException;
+import java.io.PrintStream;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Properties;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import scala.Console;
+import org.apache.zeppelin.interpreter.BaseZeppelinContext;
+import org.apache.zeppelin.interpreter.Interpreter;
+import org.apache.zeppelin.interpreter.InterpreterContext;
+import org.apache.zeppelin.interpreter.InterpreterException;
+import org.apache.zeppelin.interpreter.InterpreterOutput;
+import org.apache.zeppelin.interpreter.InterpreterResult;
+import org.apache.zeppelin.interpreter.thrift.InterpreterCompletion;
+import org.apache.zeppelin.kotlin.KotlinInterpreter;
+import org.apache.zeppelin.spark.kotlin.KotlinZeppelinBindings;
+import org.apache.zeppelin.spark.kotlin.SparkKotlinReceiver;
+
+public class KotlinSparkInterpreter extends Interpreter {
+  private static Logger logger = LoggerFactory.getLogger(KotlinSparkInterpreter.class);
+  private static final SparkVersion KOTLIN_SPARK_SUPPORTED_VERSION = SparkVersion.SPARK_2_4_0;
+
+  private InterpreterResult unsupportedMessage;
+  private KotlinInterpreter interpreter;
+  private SparkInterpreter sparkInterpreter;
+  private BaseZeppelinContext z;
+  private JavaSparkContext jsc;
+
+  public KotlinSparkInterpreter(Properties properties) {
+    super(properties);
+    logger.debug("Creating KotlinSparkInterpreter");
+    interpreter = new KotlinInterpreter(properties);
+  }
+
+  @Override
+  public void open() throws InterpreterException {
+    sparkInterpreter =
+        getInterpreterInTheSameSessionByClassName(SparkInterpreter.class);
+    jsc = sparkInterpreter.getJavaSparkContext();
+
+    SparkVersion sparkVersion = SparkVersion.fromVersionString(jsc.version());
+    if (sparkVersion.olderThan(KOTLIN_SPARK_SUPPORTED_VERSION)) {
+      unsupportedMessage = new InterpreterResult(
+          InterpreterResult.Code.ERROR,
+          "Spark version is " + sparkVersion + ", only " +
+          KOTLIN_SPARK_SUPPORTED_VERSION + " and newer are supported");
+    }
+
+    z = sparkInterpreter.getZeppelinContext();
+
+    SparkKotlinReceiver ctx = new SparkKotlinReceiver(
+        sparkInterpreter.getSparkSession(),
+        jsc,
+        sparkInterpreter.getSQLContext(),
+        z);
+
+    List<String> classpath = sparkClasspath();
+
+    String outputDir = null;
+    SparkConf conf = jsc.getConf();
+    if (conf != null) {
+      outputDir =  conf.getOption("spark.repl.class.outputDir").getOrElse(null);
+    }
+
+    interpreter.getKotlinReplProperties()
+        .receiver(ctx)
+        .classPath(classpath)
+        .outputDir(outputDir)
+        .codeOnLoad(KotlinZeppelinBindings.Z_SELECT_KOTLIN_SYNTAX)
+        .codeOnLoad(KotlinZeppelinBindings.SPARK_UDF_IMPORTS)
+        .codeOnLoad(KotlinZeppelinBindings.CAST_SPARK_SESSION);
+    interpreter.open();
+  }
+
+  @Override
+  public void close() throws InterpreterException {
+    interpreter.close();
+  }
+
+  @Override
+  public InterpreterResult interpret(String st, InterpreterContext context)
+      throws InterpreterException {
+
+    if (isSparkVersionUnsupported()) {
+      return unsupportedMessage;
+    }
+
+    z.setInterpreterContext(context);
+    z.setGui(context.getGui());
+    z.setNoteGui(context.getNoteGui());
+    InterpreterContext.set(context);
+
+    jsc.setJobGroup(buildJobGroupId(context), buildJobDesc(context), false);
+    jsc.setLocalProperty("spark.scheduler.pool", context.getLocalProperties().get("pool"));
+
+    InterpreterOutput out = context.out;
+    PrintStream scalaOut = Console.out();
+    PrintStream newOut = (out != null) ? new PrintStream(out) : null;
+
+    Console.setOut(newOut);
+    InterpreterResult result = interpreter.interpret(st, context);
+    Console.setOut(scalaOut);
+
+    return result;
+  }
+
+  @Override
+  public void cancel(InterpreterContext context) throws InterpreterException {
+    if (isSparkVersionUnsupported()) {
+      return;
+    }
+    jsc.cancelJobGroup(buildJobGroupId(context));
+    interpreter.cancel(context);
+  }
+
+  @Override
+  public FormType getFormType() throws InterpreterException {
+    return interpreter.getFormType();
+  }
+
+  @Override
+  public int getProgress(InterpreterContext context) throws InterpreterException {
+    if (isSparkVersionUnsupported()) {
+      return 0;
+    }
+    return sparkInterpreter.getProgress(context);
+  }
+
+  @Override
+  public List<InterpreterCompletion> completion(String buf, int cursor,
+      InterpreterContext interpreterContext) throws InterpreterException {
+    if (isSparkVersionUnsupported()) {
+      return Collections.emptyList();
+    }
+    return interpreter.completion(buf, cursor, interpreterContext);
+  }
+
+  boolean isSparkVersionUnsupported() {
+    return unsupportedMessage != null;
+  }
+
+  private static List<String> sparkClasspath() {
+    String sparkJars = System.getProperty("spark.jars");
+    Pattern isKotlinJar = Pattern.compile("/kotlin-[a-z]*(-.*)?\\.jar");
+
+    Stream<File> addedJars = Arrays.stream(Utils.resolveURIs(sparkJars).split(","))
+        .filter(s -> !s.trim().equals(""))
+        .filter(s -> !isKotlinJar.matcher(s).find())
+        .map(s -> {
+          int p = s.indexOf(':');
+          return new File(s.substring(p + 1));
+        });
+
+    Stream<File> systemJars = Arrays.stream(
+        System.getProperty("java.class.path").split(File.pathSeparator))
+        .map(File::new);
+
+    return Stream.concat(addedJars, systemJars)
+        .map(file -> {
+          try {
+            return file.getCanonicalPath();
+          } catch (IOException e) {
+            return "";
+          }
+        })
+        .collect(Collectors.toList());
+  }
+}
diff --git a/spark/interpreter/src/main/java/org/apache/zeppelin/spark/kotlin/KotlinZeppelinBindings.java b/spark/interpreter/src/main/java/org/apache/zeppelin/spark/kotlin/KotlinZeppelinBindings.java
new file mode 100644
index 0000000..252a936
--- /dev/null
+++ b/spark/interpreter/src/main/java/org/apache/zeppelin/spark/kotlin/KotlinZeppelinBindings.java
@@ -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.zeppelin.spark.kotlin;
+
+/**
+ * Pre-executed code on KotlinSparkInterpreter opening.
+ */
+public class KotlinZeppelinBindings {
+
+  //Simpler Kotlin syntax for z.select
+  public static final String Z_SELECT_KOTLIN_SYNTAX =
+      "import org.apache.zeppelin.display.ui.OptionInput.ParamOption\n" +
+      "import org.apache.zeppelin.interpreter.BaseZeppelinContext\n" +
+      "\n" +
+      "fun BaseZeppelinContext.select(name: String, defaultValue: Any?, " +
+          "options: List<Pair<Any?, String>>): Any? {\n" +
+      "    return select(name, defaultValue, " +
+          "options.map{ ParamOption(it.first, it.second) }.toTypedArray())\n" +
+      "}\n" +
+      "\n" +
+      "fun BaseZeppelinContext.select(name: String, options: List<Pair<Any?, String>>): Any? {\n" +
+      "    return select(name, \"\", options)\n" +
+      "}";
+
+  /**
+   * Automatic imports for Spark SQL UDFs.
+   */
+  public static final String SPARK_UDF_IMPORTS =
+      "import org.apache.spark.sql.types.DataTypes\n" +
+      "import org.apache.spark.sql.functions.*\n" +
+      "import org.apache.spark.sql.expressions.UserDefinedFunction\n" +
+      "import org.apache.spark.sql.api.java.*";
+
+  public static final String CAST_SPARK_SESSION = "" +
+      "import org.apache.spark.sql.SparkSession\n" +
+      "val spark = _sparkObject as SparkSession";
+}
diff --git a/spark/interpreter/src/main/java/org/apache/zeppelin/spark/kotlin/SparkKotlinReceiver.java b/spark/interpreter/src/main/java/org/apache/zeppelin/spark/kotlin/SparkKotlinReceiver.java
new file mode 100644
index 0000000..8d5c10b
--- /dev/null
+++ b/spark/interpreter/src/main/java/org/apache/zeppelin/spark/kotlin/SparkKotlinReceiver.java
@@ -0,0 +1,43 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES 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.zeppelin.spark.kotlin;
+
+import org.apache.spark.api.java.JavaSparkContext;
+import org.apache.spark.sql.SQLContext;
+import org.apache.zeppelin.interpreter.BaseZeppelinContext;
+import org.apache.zeppelin.kotlin.context.KotlinReceiver;
+
+/**
+ * Implicit receiver for Kotlin REPL with Spark's context (see KotlinReceiver for more details)
+ */
+public class SparkKotlinReceiver extends KotlinReceiver {
+  public final Object _sparkObject;
+  public final JavaSparkContext sc;
+  public final SQLContext sqlContext;
+  public final BaseZeppelinContext z;
+
+  public SparkKotlinReceiver(Object spark,
+                             JavaSparkContext sc,
+                             SQLContext sqlContext,
+                             BaseZeppelinContext z) {
+    this._sparkObject = spark;
+    this.sc = sc;
+    this.sqlContext = sqlContext;
+    this.z = z;
+  }
+}
diff --git a/spark/interpreter/src/main/resources/interpreter-setting.json b/spark/interpreter/src/main/resources/interpreter-setting.json
index e85509c..e4554dd 100644
--- a/spark/interpreter/src/main/resources/interpreter-setting.json
+++ b/spark/interpreter/src/main/resources/interpreter-setting.json
@@ -243,5 +243,39 @@
       "editOnDblClick": false,
       "completionSupport": false
     }
+  },
+  {
+    "group": "spark",
+    "name": "kotlin",
+    "className": "org.apache.zeppelin.spark.KotlinSparkInterpreter",
+    "properties": {
+      "zeppelin.spark.printREPLOutput": {
+        "envName": null,
+        "propertyName": "zeppelin.spark.printREPLOutput",
+        "defaultValue": true,
+        "description": "Print REPL output",
+        "type": "checkbox"
+      },
+      "zeppelin.spark.maxResult": {
+        "envName": null,
+        "propertyName": "zeppelin.kotlin.maxResult",
+        "defaultValue": "1000",
+        "description": "Max number of result to display.",
+        "type": "number"
+      },
+      "zeppelin.kotlin.shortenTypes": {
+        "envName": null,
+        "propertyName": "zeppelin.kotlin.shortenTypes",
+        "defaultValue": true,
+        "description": "Show short types instead of full, e.g. List<String> or kotlin.collections.List<kotlin.String>",
+        "type": "checkbox"
+      }
+    },
+    "editor": {
+      "language": "kotlin",
+      "editOnDblClick": false,
+      "completionKey": "TAB",
+      "completionSupport": false
+    }
   }
 ]
diff --git a/spark/interpreter/src/test/java/org/apache/zeppelin/spark/KotlinSparkInterpreterTest.java b/spark/interpreter/src/test/java/org/apache/zeppelin/spark/KotlinSparkInterpreterTest.java
new file mode 100644
index 0000000..464c8f2
--- /dev/null
+++ b/spark/interpreter/src/test/java/org/apache/zeppelin/spark/KotlinSparkInterpreterTest.java
@@ -0,0 +1,246 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES 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.zeppelin.spark;
+
+import static org.apache.zeppelin.interpreter.InterpreterResult.Code.ERROR;
+import static org.apache.zeppelin.interpreter.InterpreterResult.Code.SUCCESS;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Mockito.mock;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.Assume;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.ClassRule;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+import org.junit.rules.TemporaryFolder;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.LinkedList;
+import java.util.Properties;
+import org.apache.zeppelin.display.AngularObjectRegistry;
+import org.apache.zeppelin.display.ui.TextBox;
+import org.apache.zeppelin.interpreter.Interpreter;
+import org.apache.zeppelin.interpreter.InterpreterContext;
+import org.apache.zeppelin.interpreter.InterpreterException;
+import org.apache.zeppelin.interpreter.InterpreterGroup;
+import org.apache.zeppelin.interpreter.InterpreterOutput;
+import org.apache.zeppelin.interpreter.InterpreterOutputListener;
+import org.apache.zeppelin.interpreter.InterpreterResult;
+import org.apache.zeppelin.interpreter.InterpreterResultMessageOutput;
+import org.apache.zeppelin.interpreter.remote.RemoteInterpreterEventClient;
+import org.apache.zeppelin.resource.LocalResourcePool;
+
+public class KotlinSparkInterpreterTest {
+
+  @ClassRule
+  public static TemporaryFolder tmpDir = new TemporaryFolder();
+
+  @Rule
+  public ExpectedException exceptionRule = ExpectedException.none();
+
+  private static SparkInterpreter repl;
+  private static InterpreterGroup intpGroup;
+  private static InterpreterContext context;
+  private static KotlinSparkInterpreter interpreter;
+  private static String output;
+  private static boolean sparkSupported;
+
+  public static Properties getSparkTestProperties(TemporaryFolder tmpDir) throws IOException {
+    Properties p = new Properties();
+    p.setProperty("master", "local[*]");
+    p.setProperty("spark.app.name", "Zeppelin Test");
+    p.setProperty("zeppelin.spark.useHiveContext", "true");
+    p.setProperty("zeppelin.spark.maxResult", "1000");
+    p.setProperty("zeppelin.spark.importImplicit", "true");
+    p.setProperty("zeppelin.dep.localrepo", tmpDir.newFolder().getAbsolutePath());
+    p.setProperty("zeppelin.spark.property_1", "value_1");
+    return p;
+  }
+
+  private static void testCodeForResult(String code, String expected) throws Exception {
+    InterpreterResult result = interpreter.interpret(code, context);
+
+    String value;
+    if (result.message().isEmpty()) {
+      value = "";
+    } else {
+      String message = result.message().get(0).getData().trim();
+      // "res0 : kotlin.Int = 1" -> "kotlin.Int = 1"
+      value = message.substring(message.indexOf(':') + 2);
+    }
+
+    assertEquals(SUCCESS, result.code());
+    assertEquals(expected, value);
+  }
+
+  @BeforeClass
+  public static void setUp() throws Exception {
+    intpGroup = new InterpreterGroup();
+    context = InterpreterContext.builder()
+        .setNoteId("noteId")
+        .setParagraphId("paragraphId")
+        .setParagraphTitle("title")
+        .setAngularObjectRegistry(new AngularObjectRegistry(intpGroup.getId(), null))
+        .setResourcePool(new LocalResourcePool("id"))
+        .setInterpreterOut(new InterpreterOutput(null))
+        .setIntpEventClient(mock(RemoteInterpreterEventClient.class))
+        .build();
+    context.out = new InterpreterOutput(
+        new InterpreterOutputListener() {
+          @Override
+          public void onUpdateAll(InterpreterOutput out) {
+
+          }
+
+          @Override
+          public void onAppend(int index, InterpreterResultMessageOutput out, byte[] line) {
+            try {
+              output = out.toInterpreterResultMessage().getData();
+            } catch (IOException e) {
+              e.printStackTrace();
+            }
+          }
+
+          @Override
+          public void onUpdate(int index, InterpreterResultMessageOutput out) {
+
+          }
+        });
+
+    InterpreterContext.set(context);
+
+    intpGroup.put("note", new LinkedList<Interpreter>());
+
+    Properties properties = getSparkTestProperties(tmpDir);
+    repl = new SparkInterpreter(properties);
+    repl.setInterpreterGroup(intpGroup);
+    intpGroup.get("note").add(repl);
+    repl.open();
+    repl.interpret("sc", context);
+
+    interpreter = new KotlinSparkInterpreter(properties);
+    interpreter.setInterpreterGroup(intpGroup);
+    intpGroup.get("note").add(interpreter);
+    try {
+      interpreter.open();
+      sparkSupported = true;
+    } catch (UnsupportedClassVersionError e) {
+      sparkSupported = false;
+    }
+  }
+
+  @AfterClass
+  public static void tearDown() throws InterpreterException {
+    repl.close();
+  }
+
+  @Before
+  public void expectUnsupportedError() {
+    if (!sparkSupported) {
+      exceptionRule.expect(UnsupportedClassVersionError.class);
+    }
+    Assume.assumeFalse("Spark version should be >= 2.4.", interpreter.isSparkVersionUnsupported());
+  }
+
+  @Test
+  public void simpleKotlinTest() throws Exception {
+    testCodeForResult("1 + 1", "Int = 2");
+  }
+
+  @Test
+  public void dataFrameTest() throws Exception {
+    interpreter.interpret("spark.range(100, 0, -1).sort(\"id\").show(2)", context);
+    assertTrue(output.contains(
+        "+---+\n" +
+        "| id|\n" +
+        "+---+\n" +
+        "|  1|\n" +
+        "|  2|\n" +
+        "+---+"));
+  }
+
+  @Test
+  public void testCancel() throws Exception {
+    Thread t = new Thread(() -> {
+      try {
+        InterpreterResult result = interpreter.interpret(
+            "spark.range(10).foreach { Thread.sleep(1000) }", context);
+        assertEquals(ERROR, result.code());
+        assertTrue(result.message().get(0).getData().trim().contains("cancelled"));
+      } catch (UnsupportedClassVersionError e) {
+        if (sparkSupported) {
+          Assert.fail(e.getMessage());
+        }
+      } catch (InterpreterException e) {
+        Assert.fail(e.getMessage());
+      }
+    });
+    t.start();
+    Thread.sleep(1000);
+    interpreter.cancel(context);
+  }
+
+  @Test
+  public void sparkPropertiesTest() throws Exception {
+    InterpreterResult result = interpreter.interpret(
+        "sc.conf.all.map{ it.toString() }", context);
+    String message = result.message().get(0).getData().trim();
+    System.out.println("PROPS_1 = " + message);
+    assertTrue(message.contains("(zeppelin.spark.property_1,value_1)"));
+  }
+
+  @Test
+  public void classWriteTest() throws Exception {
+    interpreter.interpret("val f = { x: Any -> println(x) }", context);
+    output = "";
+    InterpreterResult result = interpreter.interpret("spark.range(5).foreach(f)", context);
+    assertEquals(SUCCESS, result.code());
+    assertTrue(output.contains("0"));
+    assertTrue(output.contains("1"));
+    assertTrue(output.contains("2"));
+    assertTrue(output.contains("3"));
+    assertTrue(output.contains("4"));
+
+    String classOutputDir = repl.getSparkContext().getConf().get("spark.repl.class.outputDir");
+    System.out.println(classOutputDir);
+
+    Path outPath = Paths.get(classOutputDir);
+    Files.walk(outPath).forEach(System.out::println);
+    assertTrue(Files.walk(outPath).anyMatch(path -> path.toString().matches(
+        ".*Line_\\d+\\$f\\$1\\.class")));
+    assertTrue(Files.walk(outPath).anyMatch(path -> path.toString().matches(
+        ".*Line_\\d+\\$sam\\$org_apache_spark_api_java_function_ForeachFunction\\$0\\.class")));
+  }
+
+  @Test
+  public void zeppelinContextTest() throws Exception {
+    InterpreterResult result = interpreter.interpret("z.input(\"name\", \"default_name\")", context);
+    assertEquals(InterpreterResult.Code.SUCCESS, result.code());
+    assertEquals(1, context.getGui().getForms().size());
+    assertTrue(context.getGui().getForms().get("name") instanceof TextBox);
+    TextBox textBox = (TextBox) context.getGui().getForms().get("name");
+    assertEquals("name", textBox.getName());
+    assertEquals("default_name", textBox.getDefaultValue());
+  }
+}
diff --git a/spark/spark-scala-parent/src/main/scala/org/apache/zeppelin/spark/SparkZeppelinContext.scala b/spark/spark-scala-parent/src/main/scala/org/apache/zeppelin/spark/SparkZeppelinContext.scala
index 5879962..ee0c6af 100644
--- a/spark/spark-scala-parent/src/main/scala/org/apache/zeppelin/spark/SparkZeppelinContext.scala
+++ b/spark/spark-scala-parent/src/main/scala/org/apache/zeppelin/spark/SparkZeppelinContext.scala
@@ -39,11 +39,12 @@ class SparkZeppelinContext(val sc: SparkContext,
                            val maxResult2: Int) extends BaseZeppelinContext(hooks2, maxResult2) {
 
   private val interpreterClassMap = Map(
-    ("spark", "org.apache.zeppelin.spark.SparkInterpreter"),
-    ("sql", "org.apache.zeppelin.spark.SparkSqlInterpreter"),
-    ("pyspark", "org.apache.zeppelin.spark.PySparkInterpreter"),
-    ("ipyspark", "org.apache.zeppelin.spark.IPySparkInterpreter"),
-    ("r", "org.apache.zeppelin.spark.SparkRInterpreter")
+    "spark" -> "org.apache.zeppelin.spark.SparkInterpreter",
+    "sql" -> "org.apache.zeppelin.spark.SparkSqlInterpreter",
+    "pyspark" -> "org.apache.zeppelin.spark.PySparkInterpreter",
+    "ipyspark" -> "org.apache.zeppelin.spark.IPySparkInterpreter",
+    "r" -> "org.apache.zeppelin.spark.SparkRInterpreter",
+    "kotlin" -> "org.apache.zeppelin.spark.KotlinSparkInterpreter"
   )
 
   private val supportedClasses = scala.collection.mutable.ArrayBuffer[Class[_]]()
diff --git a/zeppelin-web/bower.json b/zeppelin-web/bower.json
index b2cee2a..dac35d4 100644
--- a/zeppelin-web/bower.json
+++ b/zeppelin-web/bower.json
@@ -47,6 +47,7 @@
       "main": [
         "src-noconflict/ace.js",
         "src-noconflict/mode-scala.js",
+        "src-noconflict/mode-kotlin.js",
         "src-noconflict/mode-python.js",
         "src-noconflict/mode-sql.js",
         "src-noconflict/mode-markdown.js",
diff --git a/zeppelin-web/karma.conf.js b/zeppelin-web/karma.conf.js
index 6ceef8d..e122648 100644
--- a/zeppelin-web/karma.conf.js
+++ b/zeppelin-web/karma.conf.js
@@ -56,6 +56,7 @@ module.exports = function(config) {
       'bower_components/angular-websocket/angular-websocket.min.js',
       'bower_components/ace-builds/src-noconflict/ace.js',
       'bower_components/ace-builds/src-noconflict/mode-scala.js',
+      'bower_components/ace-builds/src-noconflict/mode-kotlin.js',
       'bower_components/ace-builds/src-noconflict/mode-python.js',
       'bower_components/ace-builds/src-noconflict/mode-sql.js',
       'bower_components/ace-builds/src-noconflict/mode-markdown.js',
diff --git a/zeppelin-web/package-lock.json b/zeppelin-web/package-lock.json
index dc32d68..a2b2537 100644
--- a/zeppelin-web/package-lock.json
+++ b/zeppelin-web/package-lock.json
@@ -96,7 +96,7 @@
       "integrity": "sha512-Mmqx/cS68K1tSrvRLtaV/Lp3NZWzXtOHUW2IvDvl2sihAwJh4ACE0eL6A8FvMyDG9abes3saB6dMimLOs+HMoQ==",
       "dev": true,
       "requires": {
-        "@xtuc/ieee754": "^1.2.0"
+        "@xtuc/ieee754": "1.2.0"
       }
     },
     "@webassemblyjs/leb128": {
@@ -211,10 +211,10 @@
       "resolved": "https://registry.npmjs.org/CSSselect/-/CSSselect-0.7.0.tgz",
       "integrity": "sha1-5AVMZ7RnRl88lQDA2gqnh4xLq9I=",
       "requires": {
-        "CSSwhat": "0.4",
-        "boolbase": "~1.0.0",
-        "domutils": "1.4",
-        "nth-check": "~1.0.0"
+        "CSSwhat": "0.4.7",
+        "boolbase": "1.0.0",
+        "domutils": "1.4.3",
+        "nth-check": "1.0.2"
       }
     },
     "CSSwhat": {
@@ -234,7 +234,7 @@
       "integrity": "sha1-63d99gEXI6OxTopywIBcjoZ0a9I=",
       "dev": true,
       "requires": {
-        "mime-types": "~2.1.18",
+        "mime-types": "2.1.21",
         "negotiator": "0.6.1"
       }
     },
@@ -250,7 +250,7 @@
       "integrity": "sha512-zVWV8Z8lislJoOKKqdNMOB+s6+XV5WERty8MnKBeFgwA+19XJjJHs2RP5dzM57FftIs+jQnRToLiWazKr6sSWg==",
       "dev": true,
       "requires": {
-        "acorn": "^5.0.0"
+        "acorn": "5.7.3"
       }
     },
     "acorn-jsx": {
@@ -259,7 +259,7 @@
       "integrity": "sha1-r9+UiPsezvyDSPb7IvRk4ypYs2s=",
       "dev": true,
       "requires": {
-        "acorn": "^3.0.4"
+        "acorn": "3.3.0"
       },
       "dependencies": {
         "acorn": {
@@ -276,19 +276,13 @@
       "integrity": "sha512-L8vcjDTCOIJk7wFvmlEUN7AsSb8T+2JrdP7KINBjzr24TJ5Mwj590sLu3BC7zNZowvJWa/JtPmD8eJCzdtDWjA==",
       "dev": true
     },
-    "after": {
-      "version": "0.8.1",
-      "resolved": "https://registry.npmjs.org/after/-/after-0.8.1.tgz",
-      "integrity": "sha1-q11PuIP1loFtNRX495HAr0ht1ic=",
-      "dev": true
-    },
     "agent-base": {
       "version": "4.2.1",
       "resolved": "https://registry.npmjs.org/agent-base/-/agent-base-4.2.1.tgz",
       "integrity": "sha512-JVwXMr9nHYTUXsBFKUqhJwvlcYU/blreOEUkhNR2eXZIvwd+c+o5V4MgDPKWnMS/56awN3TRzIP+KoPn+roQtg==",
       "dev": true,
       "requires": {
-        "es6-promisify": "^5.0.0"
+        "es6-promisify": "5.0.0"
       }
     },
     "ajv": {
@@ -297,8 +291,8 @@
       "integrity": "sha1-gv+wKynmYq5TvcIK8VlHcGc5xTY=",
       "dev": true,
       "requires": {
-        "co": "^4.6.0",
-        "json-stable-stringify": "^1.0.1"
+        "co": "4.6.0",
+        "json-stable-stringify": "1.0.1"
       }
     },
     "ajv-errors": {
@@ -325,7 +319,7 @@
       "integrity": "sha1-x1iICGF1cgNKrmJICvJrHU0cs80=",
       "dev": true,
       "requires": {
-        "stable": "~0.1.3"
+        "stable": "0.1.8"
       }
     },
     "amdefine": {
@@ -343,12 +337,11 @@
       "resolved": "https://registry.npmjs.org/angular-ui-grid/-/angular-ui-grid-4.4.6.tgz",
       "integrity": "sha512-0d14HDY4XeqFHI508thxeufiR0AlFoZQ8ihk0x8TRCQc+b9CCk1/F63W2zihirxF0cdOAqBCY2pVSM7vfZvXBQ==",
       "requires": {
-        "angular": ">=1.4.0 1.6.x"
+        "angular": "1.6.10"
       }
     },
     "angular-viewport-watch": {
-      "version": "github:wix/angular-viewport-watch#182923b3934e63817b6fc7b640ecb5c4a011f74c",
-      "from": "github:wix/angular-viewport-watch"
+      "version": "github:wix/angular-viewport-watch#182923b3934e63817b6fc7b640ecb5c4a011f74c"
     },
     "ansi-colors": {
       "version": "3.2.1",
@@ -390,9 +383,10 @@
       "resolved": "https://registry.npmjs.org/anymatch/-/anymatch-1.3.2.tgz",
       "integrity": "sha512-0XNayC8lTHQ2OI8aljNCN3sSx6hsr/1+rlcDAotXJR7C1oZZHCNsfpbKwMjRA3Uqb5tF1Rae2oloTr4xpq+WjA==",
       "dev": true,
+      "optional": true,
       "requires": {
-        "micromatch": "^2.1.5",
-        "normalize-path": "^2.0.0"
+        "micromatch": "2.3.11",
+        "normalize-path": "2.1.1"
       }
     },
     "applause": {
@@ -401,9 +395,9 @@
       "integrity": "sha1-qEaFeegfZzl7tWNMKZU77c0PVsA=",
       "dev": true,
       "requires": {
-        "cson-parser": "^1.1.0",
-        "js-yaml": "^3.3.0",
-        "lodash": "^3.10.0"
+        "cson-parser": "1.3.5",
+        "js-yaml": "3.7.0",
+        "lodash": "3.10.1"
       },
       "dependencies": {
         "lodash": {
@@ -426,7 +420,7 @@
       "integrity": "sha512-o5Roy6tNG4SL/FOkCAN6RzjiakZS25RLYFrcMttJqbdd8BWrnA+fGz57iN5Pb06pvBGvl5gQ0B48dJlslXvoTg==",
       "dev": true,
       "requires": {
-        "sprintf-js": "~1.0.2"
+        "sprintf-js": "1.0.3"
       }
     },
     "arr-diff": {
@@ -434,8 +428,9 @@
       "resolved": "https://registry.npmjs.org/arr-diff/-/arr-diff-2.0.0.tgz",
       "integrity": "sha1-jzuCf5Vai9ZpaX5KQlasPOrjVs8=",
       "dev": true,
+      "optional": true,
       "requires": {
-        "arr-flatten": "^1.0.1"
+        "arr-flatten": "1.1.0"
       }
     },
     "arr-flatten": {
@@ -492,7 +487,7 @@
       "integrity": "sha1-mjRBDk9OPaI96jdb5b5w8kd47Dk=",
       "dev": true,
       "requires": {
-        "array-uniq": "^1.0.1"
+        "array-uniq": "1.0.3"
       }
     },
     "array-uniq": {
@@ -507,12 +502,6 @@
       "integrity": "sha1-odl8yvy8JiXMcPrc6zalDFiwGlM=",
       "dev": true
     },
-    "arraybuffer.slice": {
-      "version": "0.0.6",
-      "resolved": "https://registry.npmjs.org/arraybuffer.slice/-/arraybuffer.slice-0.0.6.tgz",
-      "integrity": "sha1-8zshWfBTKj8xB6JywMz70a0peco=",
-      "dev": true
-    },
     "arrify": {
       "version": "1.0.1",
       "resolved": "https://registry.npmjs.org/arrify/-/arrify-1.0.1.tgz",
@@ -525,7 +514,7 @@
       "integrity": "sha512-jxwzQpLQjSmWXgwaCZE9Nz+glAG01yF1QnWgbhGwHI5A6FRIEY6IVqtHhIepHqI7/kyEyQEagBC5mBEFlIYvdg==",
       "dev": true,
       "requires": {
-        "safer-buffer": "~2.1.0"
+        "safer-buffer": "2.1.2"
       }
     },
     "asn1.js": {
@@ -534,9 +523,9 @@
       "integrity": "sha512-p32cOF5q0Zqs9uBiONKYLm6BClCoBCM5O9JfeUSlnQLBTxYdTK+pW+nXflm8UkKd2UYlEbYz5qEi0JuZR9ckSw==",
       "dev": true,
       "requires": {
-        "bn.js": "^4.0.0",
-        "inherits": "^2.0.1",
-        "minimalistic-assert": "^1.0.0"
+        "bn.js": "4.11.8",
+        "inherits": "2.0.3",
+        "minimalistic-assert": "1.0.1"
       }
     },
     "assert": {
@@ -588,6 +577,12 @@
       "integrity": "sha1-GdOGodntxufByF04iu28xW0zYC0=",
       "dev": true
     },
+    "async-limiter": {
+      "version": "1.0.0",
+      "resolved": "https://registry.npmjs.org/async-limiter/-/async-limiter-1.0.0.tgz",
+      "integrity": "sha512-jp/uFnooOiO+L211eZOoSyzpOITMXx1rBITauYykG3BRYPu8h0UcxsPNB04RR5vo4Tyz3+ay17tR6JVf9qzYWg==",
+      "dev": true
+    },
     "asynckit": {
       "version": "0.4.0",
       "resolved": "https://registry.npmjs.org/asynckit/-/asynckit-0.4.0.tgz",
@@ -606,12 +601,12 @@
       "integrity": "sha1-Hb0cg1ZY41zj+ZhAmdsAWFx4IBQ=",
       "dev": true,
       "requires": {
-        "browserslist": "^1.7.6",
-        "caniuse-db": "^1.0.30000634",
-        "normalize-range": "^0.1.2",
-        "num2fraction": "^1.2.2",
-        "postcss": "^5.2.16",
-        "postcss-value-parser": "^3.2.3"
+        "browserslist": "1.7.7",
+        "caniuse-db": "1.0.30000906",
+        "normalize-range": "0.1.2",
+        "num2fraction": "1.2.2",
+        "postcss": "5.2.18",
+        "postcss-value-parser": "3.3.1"
       }
     },
     "aws-sign2": {
@@ -632,21 +627,21 @@
       "integrity": "sha1-UCq1SHTX24itALiHoGODzgPQAvE=",
       "dev": true,
       "requires": {
-        "babel-core": "^6.26.0",
-        "babel-polyfill": "^6.26.0",
-        "babel-register": "^6.26.0",
-        "babel-runtime": "^6.26.0",
-        "chokidar": "^1.6.1",
-        "commander": "^2.11.0",
-        "convert-source-map": "^1.5.0",
-        "fs-readdir-recursive": "^1.0.0",
-        "glob": "^7.1.2",
-        "lodash": "^4.17.4",
-        "output-file-sync": "^1.1.2",
-        "path-is-absolute": "^1.0.1",
-        "slash": "^1.0.0",
-        "source-map": "^0.5.6",
-        "v8flags": "^2.1.1"
+        "babel-core": "6.26.3",
+        "babel-polyfill": "6.26.0",
+        "babel-register": "6.26.0",
+        "babel-runtime": "6.26.0",
+        "chokidar": "1.7.0",
+        "commander": "2.19.0",
+        "convert-source-map": "1.6.0",
+        "fs-readdir-recursive": "1.1.0",
+        "glob": "7.1.3",
+        "lodash": "4.17.11",
+        "output-file-sync": "1.1.2",
+        "path-is-absolute": "1.0.1",
+        "slash": "1.0.0",
+        "source-map": "0.5.7",
+        "v8flags": "2.1.1"
       },
       "dependencies": {
         "commander": {
@@ -661,12 +656,12 @@
           "integrity": "sha512-vcfuiIxogLV4DlGBHIUOwI0IbrJ8HWPc4MU7HzviGeNho/UJDfi6B5p3sHeWIQ0KGIU0Jpxi5ZHxemQfLkkAwQ==",
           "dev": true,
           "requires": {
-            "fs.realpath": "^1.0.0",
-            "inflight": "^1.0.4",
-            "inherits": "2",
-            "minimatch": "^3.0.4",
-            "once": "^1.3.0",
-            "path-is-absolute": "^1.0.0"
+            "fs.realpath": "1.0.0",
+            "inflight": "1.0.6",
+            "inherits": "2.0.3",
+            "minimatch": "3.0.4",
+            "once": "1.4.0",
+            "path-is-absolute": "1.0.1"
           }
         },
         "minimatch": {
@@ -675,7 +670,7 @@
           "integrity": "sha512-yJHVQEhyqPLUTgt9B83PXu6W3rx4MvvHvSUvToogpwoGDOUQ+yDrR0HRot+yOCdCO7u4hX3pWft6kWBBcqh0UA==",
           "dev": true,
           "requires": {
-            "brace-expansion": "^1.1.7"
+            "brace-expansion": "1.1.11"
           }
         },
         "source-map": {
@@ -692,9 +687,9 @@
       "integrity": "sha1-Y/1D99weO7fONZR9uP42mj9Yx0s=",
       "dev": true,
       "requires": {
-        "chalk": "^1.1.3",
-        "esutils": "^2.0.2",
-        "js-tokens": "^3.0.2"
+        "chalk": "1.1.3",
+        "esutils": "2.0.2",
+        "js-tokens": "3.0.2"
       }
     },
     "babel-core": {
@@ -703,25 +698,25 @@
       "integrity": "sha512-6jyFLuDmeidKmUEb3NM+/yawG0M2bDZ9Z1qbZP59cyHLz8kYGKYwpJP0UwUKKUiTRNvxfLesJnTedqczP7cTDA==",
       "dev": true,
       "requires": {
-        "babel-code-frame": "^6.26.0",
-        "babel-generator": "^6.26.0",
-        "babel-helpers": "^6.24.1",
-        "babel-messages": "^6.23.0",
-        "babel-register": "^6.26.0",
-        "babel-runtime": "^6.26.0",
-        "babel-template": "^6.26.0",
-        "babel-traverse": "^6.26.0",
-        "babel-types": "^6.26.0",
-        "babylon": "^6.18.0",
-        "convert-source-map": "^1.5.1",
-        "debug": "^2.6.9",
-        "json5": "^0.5.1",
-        "lodash": "^4.17.4",
-        "minimatch": "^3.0.4",
-        "path-is-absolute": "^1.0.1",
-        "private": "^0.1.8",
-        "slash": "^1.0.0",
-        "source-map": "^0.5.7"
+        "babel-code-frame": "6.26.0",
+        "babel-generator": "6.26.1",
+        "babel-helpers": "6.24.1",
+        "babel-messages": "6.23.0",
+        "babel-register": "6.26.0",
+        "babel-runtime": "6.26.0",
+        "babel-template": "6.26.0",
+        "babel-traverse": "6.26.0",
+        "babel-types": "6.26.0",
+        "babylon": "6.18.0",
+        "convert-source-map": "1.6.0",
+        "debug": "2.6.9",
+        "json5": "0.5.1",
+        "lodash": "4.17.11",
+        "minimatch": "3.0.4",
+        "path-is-absolute": "1.0.1",
+        "private": "0.1.8",
+        "slash": "1.0.0",
+        "source-map": "0.5.7"
       },
       "dependencies": {
         "minimatch": {
@@ -730,7 +725,7 @@
           "integrity": "sha512-yJHVQEhyqPLUTgt9B83PXu6W3rx4MvvHvSUvToogpwoGDOUQ+yDrR0HRot+yOCdCO7u4hX3pWft6kWBBcqh0UA==",
           "dev": true,
           "requires": {
-            "brace-expansion": "^1.1.7"
+            "brace-expansion": "1.1.11"
           }
         },
         "source-map": {
@@ -747,14 +742,14 @@
       "integrity": "sha512-HyfwY6ApZj7BYTcJURpM5tznulaBvyio7/0d4zFOeMPUmfxkCjHocCuoLa2SAGzBI8AREcH3eP3758F672DppA==",
       "dev": true,
       "requires": {
-        "babel-messages": "^6.23.0",
-        "babel-runtime": "^6.26.0",
-        "babel-types": "^6.26.0",
-        "detect-indent": "^4.0.0",
-        "jsesc": "^1.3.0",
-        "lodash": "^4.17.4",
-        "source-map": "^0.5.7",
-        "trim-right": "^1.0.1"
+        "babel-messages": "6.23.0",
+        "babel-runtime": "6.26.0",
+        "babel-types": "6.26.0",
+        "detect-indent": "4.0.0",
+        "jsesc": "1.3.0",
+        "lodash": "4.17.11",
+        "source-map": "0.5.7",
+        "trim-right": "1.0.1"
       },
       "dependencies": {
         "source-map": {
@@ -771,9 +766,9 @@
       "integrity": "sha1-zORReto1b0IgvK6KAsKzRvmlZmQ=",
       "dev": true,
       "requires": {
-        "babel-helper-explode-assignable-expression": "^6.24.1",
-        "babel-runtime": "^6.22.0",
-        "babel-types": "^6.24.1"
+        "babel-helper-explode-assignable-expression": "6.24.1",
+        "babel-runtime": "6.26.0",
+        "babel-types": "6.26.0"
       }
     },
     "babel-helper-call-delegate": {
@@ -782,10 +777,10 @@
       "integrity": "sha1-7Oaqzdx25Bw0YfiL/Fdb0Nqi340=",
       "dev": true,
       "requires": {
-        "babel-helper-hoist-variables": "^6.24.1",
-        "babel-runtime": "^6.22.0",
-        "babel-traverse": "^6.24.1",
-        "babel-types": "^6.24.1"
+        "babel-helper-hoist-variables": "6.24.1",
+        "babel-runtime": "6.26.0",
+        "babel-traverse": "6.26.0",
+        "babel-types": "6.26.0"
       }
     },
     "babel-helper-define-map": {
@@ -794,10 +789,10 @@
       "integrity": "sha1-pfVtq0GiX5fstJjH66ypgZ+Vvl8=",
       "dev": true,
       "requires": {
-        "babel-helper-function-name": "^6.24.1",
-        "babel-runtime": "^6.26.0",
-        "babel-types": "^6.26.0",
-        "lodash": "^4.17.4"
+        "babel-helper-function-name": "6.24.1",
+        "babel-runtime": "6.26.0",
+        "babel-types": "6.26.0",
+        "lodash": "4.17.11"
       }
     },
     "babel-helper-explode-assignable-expression": {
@@ -806,9 +801,9 @@
       "integrity": "sha1-8luCz33BBDPFX3BZLVdGQArCLKo=",
       "dev": true,
       "requires": {
-        "babel-runtime": "^6.22.0",
-        "babel-traverse": "^6.24.1",
-        "babel-types": "^6.24.1"
+        "babel-runtime": "6.26.0",
+        "babel-traverse": "6.26.0",
+        "babel-types": "6.26.0"
       }
     },
     "babel-helper-function-name": {
@@ -817,11 +812,11 @@
       "integrity": "sha1-00dbjAPtmCQqJbSDUasYOZ01gKk=",
       "dev": true,
       "requires": {
-        "babel-helper-get-function-arity": "^6.24.1",
-        "babel-runtime": "^6.22.0",
-        "babel-template": "^6.24.1",
-        "babel-traverse": "^6.24.1",
-        "babel-types": "^6.24.1"
+        "babel-helper-get-function-arity": "6.24.1",
+        "babel-runtime": "6.26.0",
+        "babel-template": "6.26.0",
+        "babel-traverse": "6.26.0",
+        "babel-types": "6.26.0"
       }
     },
     "babel-helper-get-function-arity": {
@@ -830,8 +825,8 @@
       "integrity": "sha1-j3eCqpNAfEHTqlCQj4mwMbG2hT0=",
       "dev": true,
       "requires": {
-        "babel-runtime": "^6.22.0",
-        "babel-types": "^6.24.1"
+        "babel-runtime": "6.26.0",
+        "babel-types": "6.26.0"
       }
     },
     "babel-helper-hoist-variables": {
@@ -840,8 +835,8 @@
       "integrity": "sha1-HssnaJydJVE+rbyZFKc/VAi+enY=",
       "dev": true,
       "requires": {
-        "babel-runtime": "^6.22.0",
-        "babel-types": "^6.24.1"
+        "babel-runtime": "6.26.0",
+        "babel-types": "6.26.0"
       }
     },
     "babel-helper-optimise-call-expression": {
@@ -850,8 +845,8 @@
       "integrity": "sha1-96E0J7qfc/j0+pk8VKl4gtEkQlc=",
       "dev": true,
       "requires": {
-        "babel-runtime": "^6.22.0",
-        "babel-types": "^6.24.1"
+        "babel-runtime": "6.26.0",
+        "babel-types": "6.26.0"
       }
     },
     "babel-helper-regex": {
@@ -860,9 +855,9 @@
       "integrity": "sha1-MlxZ+QL4LyS3T6zu0DY5VPZJXnI=",
       "dev": true,
       "requires": {
-        "babel-runtime": "^6.26.0",
-        "babel-types": "^6.26.0",
-        "lodash": "^4.17.4"
+        "babel-runtime": "6.26.0",
+        "babel-types": "6.26.0",
+        "lodash": "4.17.11"
       }
     },
     "babel-helper-remap-async-to-generator": {
@@ -871,11 +866,11 @@
       "integrity": "sha1-XsWBgnrXI/7N04HxySg5BnbkVRs=",
       "dev": true,
       "requires": {
-        "babel-helper-function-name": "^6.24.1",
-        "babel-runtime": "^6.22.0",
-        "babel-template": "^6.24.1",
-        "babel-traverse": "^6.24.1",
-        "babel-types": "^6.24.1"
+        "babel-helper-function-name": "6.24.1",
+        "babel-runtime": "6.26.0",
+        "babel-template": "6.26.0",
+        "babel-traverse": "6.26.0",
+        "babel-types": "6.26.0"
       }
     },
     "babel-helper-replace-supers": {
@@ -884,12 +879,12 @@
       "integrity": "sha1-v22/5Dk40XNpohPKiov3S2qQqxo=",
       "dev": true,
       "requires": {
-        "babel-helper-optimise-call-expression": "^6.24.1",
-        "babel-messages": "^6.23.0",
-        "babel-runtime": "^6.22.0",
-        "babel-template": "^6.24.1",
-        "babel-traverse": "^6.24.1",
-        "babel-types": "^6.24.1"
+        "babel-helper-optimise-call-expression": "6.24.1",
+        "babel-messages": "6.23.0",
+        "babel-runtime": "6.26.0",
+        "babel-template": "6.26.0",
+        "babel-traverse": "6.26.0",
+        "babel-types": "6.26.0"
       }
     },
     "babel-helpers": {
@@ -898,8 +893,8 @@
       "integrity": "sha1-NHHenK7DiOXIUOWX5Yom3fN2ArI=",
       "dev": true,
       "requires": {
-        "babel-runtime": "^6.22.0",
-        "babel-template": "^6.24.1"
+        "babel-runtime": "6.26.0",
+        "babel-template": "6.26.0"
       }
     },
     "babel-loader": {
@@ -908,9 +903,9 @@
       "integrity": "sha512-iCHfbieL5d1LfOQeeVJEUyD9rTwBcP/fcEbRCfempxTDuqrKpu0AZjLAQHEQa3Yqyj9ORKe2iHfoj4rHLf7xpw==",
       "dev": true,
       "requires": {
-        "find-cache-dir": "^1.0.0",
-        "loader-utils": "^1.0.2",
-        "mkdirp": "^0.5.1"
+        "find-cache-dir": "1.0.0",
+        "loader-utils": "1.1.0",
+        "mkdirp": "0.5.1"
       }
     },
     "babel-messages": {
@@ -919,7 +914,7 @@
       "integrity": "sha1-8830cDhYA1sqKVHG7F7fbGLyYw4=",
       "dev": true,
       "requires": {
-        "babel-runtime": "^6.22.0"
+        "babel-runtime": "6.26.0"
       }
     },
     "babel-plugin-check-es2015-constants": {
@@ -928,7 +923,7 @@
       "integrity": "sha1-NRV7EBQm/S/9PaP3XH0ekYNbv4o=",
       "dev": true,
       "requires": {
-        "babel-runtime": "^6.22.0"
+        "babel-runtime": "6.26.0"
       }
     },
     "babel-plugin-syntax-async-functions": {
@@ -955,9 +950,9 @@
       "integrity": "sha1-ZTbjeK/2yx1VF6wOQOs+n8jQh2E=",
       "dev": true,
       "requires": {
-        "babel-helper-remap-async-to-generator": "^6.24.1",
-        "babel-plugin-syntax-async-functions": "^6.8.0",
-        "babel-runtime": "^6.22.0"
+        "babel-helper-remap-async-to-generator": "6.24.1",
+        "babel-plugin-syntax-async-functions": "6.13.0",
+        "babel-runtime": "6.26.0"
       }
     },
     "babel-plugin-transform-es2015-arrow-functions": {
@@ -966,7 +961,7 @@
       "integrity": "sha1-RSaSy3EdX3ncf4XkQM5BufJE0iE=",
       "dev": true,
       "requires": {
-        "babel-runtime": "^6.22.0"
+        "babel-runtime": "6.26.0"
       }
     },
     "babel-plugin-transform-es2015-block-scoped-functions": {
@@ -975,7 +970,7 @@
       "integrity": "sha1-u8UbSflk1wy42OC5ToICRs46YUE=",
       "dev": true,
       "requires": {
-        "babel-runtime": "^6.22.0"
+        "babel-runtime": "6.26.0"
       }
     },
     "babel-plugin-transform-es2015-block-scoping": {
@@ -984,11 +979,11 @@
       "integrity": "sha1-1w9SmcEwjQXBL0Y4E7CgnnOxiV8=",
       "dev": true,
       "requires": {
-        "babel-runtime": "^6.26.0",
-        "babel-template": "^6.26.0",
-        "babel-traverse": "^6.26.0",
-        "babel-types": "^6.26.0",
-        "lodash": "^4.17.4"
+        "babel-runtime": "6.26.0",
+        "babel-template": "6.26.0",
+        "babel-traverse": "6.26.0",
+        "babel-types": "6.26.0",
+        "lodash": "4.17.11"
       }
     },
     "babel-plugin-transform-es2015-classes": {
@@ -997,15 +992,15 @@
       "integrity": "sha1-WkxYpQyclGHlZLSyo7+ryXolhNs=",
       "dev": true,
       "requires": {
-        "babel-helper-define-map": "^6.24.1",
-        "babel-helper-function-name": "^6.24.1",
-        "babel-helper-optimise-call-expression": "^6.24.1",
-        "babel-helper-replace-supers": "^6.24.1",
-        "babel-messages": "^6.23.0",
-        "babel-runtime": "^6.22.0",
-        "babel-template": "^6.24.1",
-        "babel-traverse": "^6.24.1",
-        "babel-types": "^6.24.1"
+        "babel-helper-define-map": "6.26.0",
+        "babel-helper-function-name": "6.24.1",
+        "babel-helper-optimise-call-expression": "6.24.1",
+        "babel-helper-replace-supers": "6.24.1",
+        "babel-messages": "6.23.0",
+        "babel-runtime": "6.26.0",
+        "babel-template": "6.26.0",
+        "babel-traverse": "6.26.0",
+        "babel-types": "6.26.0"
       }
     },
     "babel-plugin-transform-es2015-computed-properties": {
@@ -1014,8 +1009,8 @@
       "integrity": "sha1-b+Ko0WiV1WNPTNmZttNICjCBWbM=",
       "dev": true,
       "requires": {
-        "babel-runtime": "^6.22.0",
-        "babel-template": "^6.24.1"
+        "babel-runtime": "6.26.0",
+        "babel-template": "6.26.0"
       }
     },
     "babel-plugin-transform-es2015-destructuring": {
@@ -1024,7 +1019,7 @@
       "integrity": "sha1-mXux8auWf2gtKwh2/jWNYOdlxW0=",
       "dev": true,
       "requires": {
-        "babel-runtime": "^6.22.0"
+        "babel-runtime": "6.26.0"
       }
     },
     "babel-plugin-transform-es2015-duplicate-keys": {
@@ -1033,8 +1028,8 @@
       "integrity": "sha1-c+s9MQypaePvnskcU3QabxV2Qj4=",
       "dev": true,
       "requires": {
-        "babel-runtime": "^6.22.0",
-        "babel-types": "^6.24.1"
+        "babel-runtime": "6.26.0",
+        "babel-types": "6.26.0"
       }
     },
     "babel-plugin-transform-es2015-for-of": {
@@ -1043,7 +1038,7 @@
       "integrity": "sha1-9HyVsrYT3x0+zC/bdXNiPHUkhpE=",
       "dev": true,
       "requires": {
-        "babel-runtime": "^6.22.0"
+        "babel-runtime": "6.26.0"
       }
     },
     "babel-plugin-transform-es2015-function-name": {
@@ -1052,9 +1047,9 @@
       "integrity": "sha1-g0yJhTvDaxrw86TF26qU/Y6sqos=",
       "dev": true,
       "requires": {
-        "babel-helper-function-name": "^6.24.1",
-        "babel-runtime": "^6.22.0",
-        "babel-types": "^6.24.1"
+        "babel-helper-function-name": "6.24.1",
+        "babel-runtime": "6.26.0",
+        "babel-types": "6.26.0"
       }
     },
     "babel-plugin-transform-es2015-literals": {
@@ -1063,7 +1058,7 @@
       "integrity": "sha1-T1SgLWzWbPkVKAAZox0xklN3yi4=",
       "dev": true,
       "requires": {
-        "babel-runtime": "^6.22.0"
+        "babel-runtime": "6.26.0"
       }
     },
     "babel-plugin-transform-es2015-modules-amd": {
@@ -1072,9 +1067,9 @@
       "integrity": "sha1-Oz5UAXI5hC1tGcMBHEvS8AoA0VQ=",
       "dev": true,
       "requires": {
-        "babel-plugin-transform-es2015-modules-commonjs": "^6.24.1",
-        "babel-runtime": "^6.22.0",
-        "babel-template": "^6.24.1"
+        "babel-plugin-transform-es2015-modules-commonjs": "6.26.2",
+        "babel-runtime": "6.26.0",
+        "babel-template": "6.26.0"
       }
     },
     "babel-plugin-transform-es2015-modules-commonjs": {
@@ -1083,10 +1078,10 @@
       "integrity": "sha512-CV9ROOHEdrjcwhIaJNBGMBCodN+1cfkwtM1SbUHmvyy35KGT7fohbpOxkE2uLz1o6odKK2Ck/tz47z+VqQfi9Q==",
       "dev": true,
       "requires": {
-        "babel-plugin-transform-strict-mode": "^6.24.1",
-        "babel-runtime": "^6.26.0",
-        "babel-template": "^6.26.0",
-        "babel-types": "^6.26.0"
+        "babel-plugin-transform-strict-mode": "6.24.1",
+        "babel-runtime": "6.26.0",
+        "babel-template": "6.26.0",
+        "babel-types": "6.26.0"
       }
     },
     "babel-plugin-transform-es2015-modules-systemjs": {
@@ -1095,9 +1090,9 @@
       "integrity": "sha1-/4mhQrkRmpBhlfXxBuzzBdlAfSM=",
       "dev": true,
       "requires": {
-        "babel-helper-hoist-variables": "^6.24.1",
-        "babel-runtime": "^6.22.0",
-        "babel-template": "^6.24.1"
+        "babel-helper-hoist-variables": "6.24.1",
+        "babel-runtime": "6.26.0",
+        "babel-template": "6.26.0"
       }
     },
     "babel-plugin-transform-es2015-modules-umd": {
@@ -1106,9 +1101,9 @@
       "integrity": "sha1-rJl+YoXNGO1hdq22B9YCNErThGg=",
       "dev": true,
       "requires": {
-        "babel-plugin-transform-es2015-modules-amd": "^6.24.1",
-        "babel-runtime": "^6.22.0",
-        "babel-template": "^6.24.1"
+        "babel-plugin-transform-es2015-modules-amd": "6.24.1",
+        "babel-runtime": "6.26.0",
+        "babel-template": "6.26.0"
       }
     },
     "babel-plugin-transform-es2015-object-super": {
@@ -1117,8 +1112,8 @@
       "integrity": "sha1-JM72muIcuDp/hgPa0CH1cusnj40=",
       "dev": true,
       "requires": {
-        "babel-helper-replace-supers": "^6.24.1",
-        "babel-runtime": "^6.22.0"
+        "babel-helper-replace-supers": "6.24.1",
+        "babel-runtime": "6.26.0"
       }
     },
     "babel-plugin-transform-es2015-parameters": {
@@ -1127,12 +1122,12 @@
       "integrity": "sha1-V6w1GrScrxSpfNE7CfZv3wpiXys=",
       "dev": true,
       "requires": {
-        "babel-helper-call-delegate": "^6.24.1",
-        "babel-helper-get-function-arity": "^6.24.1",
-        "babel-runtime": "^6.22.0",
-        "babel-template": "^6.24.1",
-        "babel-traverse": "^6.24.1",
-        "babel-types": "^6.24.1"
+        "babel-helper-call-delegate": "6.24.1",
+        "babel-helper-get-function-arity": "6.24.1",
+        "babel-runtime": "6.26.0",
+        "babel-template": "6.26.0",
+        "babel-traverse": "6.26.0",
+        "babel-types": "6.26.0"
       }
     },
     "babel-plugin-transform-es2015-shorthand-properties": {
@@ -1141,8 +1136,8 @@
       "integrity": "sha1-JPh11nIch2YbvZmkYi5R8U3jiqA=",
       "dev": true,
       "requires": {
-        "babel-runtime": "^6.22.0",
-        "babel-types": "^6.24.1"
+        "babel-runtime": "6.26.0",
+        "babel-types": "6.26.0"
       }
     },
     "babel-plugin-transform-es2015-spread": {
@@ -1151,7 +1146,7 @@
       "integrity": "sha1-1taKmfia7cRTbIGlQujdnxdG+NE=",
       "dev": true,
       "requires": {
-        "babel-runtime": "^6.22.0"
+        "babel-runtime": "6.26.0"
       }
     },
     "babel-plugin-transform-es2015-sticky-regex": {
@@ -1160,9 +1155,9 @@
       "integrity": "sha1-AMHNsaynERLN8M9hJsLta0V8zbw=",
       "dev": true,
       "requires": {
-        "babel-helper-regex": "^6.24.1",
-        "babel-runtime": "^6.22.0",
-        "babel-types": "^6.24.1"
+        "babel-helper-regex": "6.26.0",
+        "babel-runtime": "6.26.0",
+        "babel-types": "6.26.0"
       }
     },
     "babel-plugin-transform-es2015-template-literals": {
@@ -1171,7 +1166,7 @@
       "integrity": "sha1-qEs0UPfp+PH2g51taH2oS7EjbY0=",
       "dev": true,
       "requires": {
-        "babel-runtime": "^6.22.0"
+        "babel-runtime": "6.26.0"
       }
     },
     "babel-plugin-transform-es2015-typeof-symbol": {
@@ -1180,7 +1175,7 @@
       "integrity": "sha1-3sCfHN3/lLUqxz1QXITfWdzOs3I=",
       "dev": true,
       "requires": {
-        "babel-runtime": "^6.22.0"
+        "babel-runtime": "6.26.0"
       }
     },
     "babel-plugin-transform-es2015-unicode-regex": {
@@ -1189,9 +1184,9 @@
       "integrity": "sha1-04sS9C6nMj9yk4fxinxa4frrNek=",
       "dev": true,
       "requires": {
-        "babel-helper-regex": "^6.24.1",
-        "babel-runtime": "^6.22.0",
-        "regexpu-core": "^2.0.0"
+        "babel-helper-regex": "6.26.0",
+        "babel-runtime": "6.26.0",
+        "regexpu-core": "2.0.0"
       }
     },
     "babel-plugin-transform-exponentiation-operator": {
@@ -1200,9 +1195,9 @@
       "integrity": "sha1-KrDJx/MJj6SJB3cruBP+QejeOg4=",
       "dev": true,
       "requires": {
-        "babel-helper-builder-binary-assignment-operator-visitor": "^6.24.1",
-        "babel-plugin-syntax-exponentiation-operator": "^6.8.0",
-        "babel-runtime": "^6.22.0"
+        "babel-helper-builder-binary-assignment-operator-visitor": "6.24.1",
+        "babel-plugin-syntax-exponentiation-operator": "6.13.0",
+        "babel-runtime": "6.26.0"
       }
     },
     "babel-plugin-transform-regenerator": {
@@ -1211,7 +1206,7 @@
       "integrity": "sha1-4HA2lvveJ/Cj78rPi03KL3s6jy8=",
       "dev": true,
       "requires": {
-        "regenerator-transform": "^0.10.0"
+        "regenerator-transform": "0.10.1"
       }
     },
     "babel-plugin-transform-strict-mode": {
@@ -1220,8 +1215,8 @@
       "integrity": "sha1-1fr3qleKZbvlkc9e2uBKDGcCB1g=",
       "dev": true,
       "requires": {
-        "babel-runtime": "^6.22.0",
-        "babel-types": "^6.24.1"
+        "babel-runtime": "6.26.0",
+        "babel-types": "6.26.0"
       }
     },
     "babel-polyfill": {
@@ -1230,9 +1225,9 @@
       "integrity": "sha1-N5k3q8Z9eJWXCtxiHyhM2WbPIVM=",
       "dev": true,
       "requires": {
-        "babel-runtime": "^6.26.0",
-        "core-js": "^2.5.0",
-        "regenerator-runtime": "^0.10.5"
+        "babel-runtime": "6.26.0",
+        "core-js": "2.5.7",
+        "regenerator-runtime": "0.10.5"
       },
       "dependencies": {
         "regenerator-runtime": {
@@ -1249,36 +1244,36 @@
       "integrity": "sha512-9OR2afuKDneX2/q2EurSftUYM0xGu4O2D9adAhVfADDhrYDaxXV0rBbevVYoY9n6nyX1PmQW/0jtpJvUNr9CHg==",
       "dev": true,
       "requires": {
-        "babel-plugin-check-es2015-constants": "^6.22.0",
-        "babel-plugin-syntax-trailing-function-commas": "^6.22.0",
-        "babel-plugin-transform-async-to-generator": "^6.22.0",
-        "babel-plugin-transform-es2015-arrow-functions": "^6.22.0",
-        "babel-plugin-transform-es2015-block-scoped-functions": "^6.22.0",
-        "babel-plugin-transform-es2015-block-scoping": "^6.23.0",
-        "babel-plugin-transform-es2015-classes": "^6.23.0",
-        "babel-plugin-transform-es2015-computed-properties": "^6.22.0",
-        "babel-plugin-transform-es2015-destructuring": "^6.23.0",
-        "babel-plugin-transform-es2015-duplicate-keys": "^6.22.0",
-        "babel-plugin-transform-es2015-for-of": "^6.23.0",
-        "babel-plugin-transform-es2015-function-name": "^6.22.0",
-        "babel-plugin-transform-es2015-literals": "^6.22.0",
-        "babel-plugin-transform-es2015-modules-amd": "^6.22.0",
-        "babel-plugin-transform-es2015-modules-commonjs": "^6.23.0",
-        "babel-plugin-transform-es2015-modules-systemjs": "^6.23.0",
-        "babel-plugin-transform-es2015-modules-umd": "^6.23.0",
-        "babel-plugin-transform-es2015-object-super": "^6.22.0",
-        "babel-plugin-transform-es2015-parameters": "^6.23.0",
-        "babel-plugin-transform-es2015-shorthand-properties": "^6.22.0",
-        "babel-plugin-transform-es2015-spread": "^6.22.0",
-        "babel-plugin-transform-es2015-sticky-regex": "^6.22.0",
-        "babel-plugin-transform-es2015-template-literals": "^6.22.0",
-        "babel-plugin-transform-es2015-typeof-symbol": "^6.23.0",
-        "babel-plugin-transform-es2015-unicode-regex": "^6.22.0",
-        "babel-plugin-transform-exponentiation-operator": "^6.22.0",
-        "babel-plugin-transform-regenerator": "^6.22.0",
-        "browserslist": "^3.2.6",
-        "invariant": "^2.2.2",
-        "semver": "^5.3.0"
+        "babel-plugin-check-es2015-constants": "6.22.0",
+        "babel-plugin-syntax-trailing-function-commas": "6.22.0",
+        "babel-plugin-transform-async-to-generator": "6.24.1",
+        "babel-plugin-transform-es2015-arrow-functions": "6.22.0",
+        "babel-plugin-transform-es2015-block-scoped-functions": "6.22.0",
+        "babel-plugin-transform-es2015-block-scoping": "6.26.0",
+        "babel-plugin-transform-es2015-classes": "6.24.1",
+        "babel-plugin-transform-es2015-computed-properties": "6.24.1",
+        "babel-plugin-transform-es2015-destructuring": "6.23.0",
+        "babel-plugin-transform-es2015-duplicate-keys": "6.24.1",
+        "babel-plugin-transform-es2015-for-of": "6.23.0",
+        "babel-plugin-transform-es2015-function-name": "6.24.1",
+        "babel-plugin-transform-es2015-literals": "6.22.0",
+        "babel-plugin-transform-es2015-modules-amd": "6.24.1",
+        "babel-plugin-transform-es2015-modules-commonjs": "6.26.2",
+        "babel-plugin-transform-es2015-modules-systemjs": "6.24.1",
+        "babel-plugin-transform-es2015-modules-umd": "6.24.1",
+        "babel-plugin-transform-es2015-object-super": "6.24.1",
+        "babel-plugin-transform-es2015-parameters": "6.24.1",
+        "babel-plugin-transform-es2015-shorthand-properties": "6.24.1",
+        "babel-plugin-transform-es2015-spread": "6.22.0",
+        "babel-plugin-transform-es2015-sticky-regex": "6.24.1",
+        "babel-plugin-transform-es2015-template-literals": "6.22.0",
+        "babel-plugin-transform-es2015-typeof-symbol": "6.23.0",
+        "babel-plugin-transform-es2015-unicode-regex": "6.24.1",
+        "babel-plugin-transform-exponentiation-operator": "6.24.1",
+        "babel-plugin-transform-regenerator": "6.26.0",
+        "browserslist": "3.2.8",
+        "invariant": "2.2.4",
+        "semver": "5.6.0"
       },
       "dependencies": {
         "browserslist": {
@@ -1287,8 +1282,8 @@
           "integrity": "sha512-WHVocJYavUwVgVViC0ORikPHQquXwVh939TaelZ4WDqpWgTX/FsGhl/+P4qBUAGcRvtOgDgC+xftNWWp2RUTAQ==",
           "dev": true,
           "requires": {
-            "caniuse-lite": "^1.0.30000844",
-            "electron-to-chromium": "^1.3.47"
+            "caniuse-lite": "1.0.30000906",
+            "electron-to-chromium": "1.3.83"
           }
         }
       }
@@ -1299,13 +1294,13 @@
       "integrity": "sha1-btAhFz4vy0htestFxgCahW9kcHE=",
       "dev": true,
       "requires": {
-        "babel-core": "^6.26.0",
-        "babel-runtime": "^6.26.0",
-        "core-js": "^2.5.0",
-        "home-or-tmp": "^2.0.0",
-        "lodash": "^4.17.4",
-        "mkdirp": "^0.5.1",
-        "source-map-support": "^0.4.15"
+        "babel-core": "6.26.3",
+        "babel-runtime": "6.26.0",
+        "core-js": "2.5.7",
+        "home-or-tmp": "2.0.0",
+        "lodash": "4.17.11",
+        "mkdirp": "0.5.1",
+        "source-map-support": "0.4.18"
       }
     },
     "babel-runtime": {
@@ -1314,8 +1309,8 @@
       "integrity": "sha1-llxwWGaOgrVde/4E/yM3vItWR/4=",
       "dev": true,
       "requires": {
-        "core-js": "^2.4.0",
-        "regenerator-runtime": "^0.11.0"
+        "core-js": "2.5.7",
+        "regenerator-runtime": "0.11.1"
       }
     },
     "babel-template": {
@@ -1324,11 +1319,11 @@
       "integrity": "sha1-3gPi0WOWsGn0bdn/+FIfsaDjXgI=",
       "dev": true,
       "requires": {
-        "babel-runtime": "^6.26.0",
-        "babel-traverse": "^6.26.0",
-        "babel-types": "^6.26.0",
-        "babylon": "^6.18.0",
-        "lodash": "^4.17.4"
+        "babel-runtime": "6.26.0",
+        "babel-traverse": "6.26.0",
+        "babel-types": "6.26.0",
+        "babylon": "6.18.0",
+        "lodash": "4.17.11"
       }
     },
     "babel-traverse": {
@@ -1337,15 +1332,15 @@
       "integrity": "sha1-RqnL1+3MYsjlwGTi0tjQ9ANXZu4=",
       "dev": true,
       "requires": {
-        "babel-code-frame": "^6.26.0",
-        "babel-messages": "^6.23.0",
-        "babel-runtime": "^6.26.0",
-        "babel-types": "^6.26.0",
-        "babylon": "^6.18.0",
-        "debug": "^2.6.8",
-        "globals": "^9.18.0",
-        "invariant": "^2.2.2",
-        "lodash": "^4.17.4"
+        "babel-code-frame": "6.26.0",
+        "babel-messages": "6.23.0",
+        "babel-runtime": "6.26.0",
+        "babel-types": "6.26.0",
+        "babylon": "6.18.0",
+        "debug": "2.6.9",
+        "globals": "9.18.0",
+        "invariant": "2.2.4",
+        "lodash": "4.17.11"
       }
     },
     "babel-types": {
@@ -1354,10 +1349,10 @@
       "integrity": "sha1-o7Bz+Uq0nrb6Vc1lInozQ4BjJJc=",
       "dev": true,
       "requires": {
-        "babel-runtime": "^6.26.0",
-        "esutils": "^2.0.2",
-        "lodash": "^4.17.4",
-        "to-fast-properties": "^1.0.3"
+        "babel-runtime": "6.26.0",
+        "esutils": "2.0.2",
+        "lodash": "4.17.11",
+        "to-fast-properties": "1.0.3"
       }
     },
     "babylon": {
@@ -1384,13 +1379,13 @@
       "integrity": "sha512-5T6P4xPgpp0YDFvSWwEZ4NoE3aM4QBQXDzmVbraCkFj8zHM+mba8SyqB5DbZWyR7mYHo6Y7BdQo3MoA4m0TeQg==",
       "dev": true,
       "requires": {
-        "cache-base": "^1.0.1",
-        "class-utils": "^0.3.5",
-        "component-emitter": "^1.2.1",
-        "define-property": "^1.0.0",
-        "isobject": "^3.0.1",
-        "mixin-deep": "^1.2.0",
-        "pascalcase": "^0.1.1"
+        "cache-base": "1.0.1",
+        "class-utils": "0.3.6",
+        "component-emitter": "1.2.1",
+        "define-property": "1.0.0",
+        "isobject": "3.0.1",
+        "mixin-deep": "1.3.1",
+        "pascalcase": "0.1.1"
       },
       "dependencies": {
         "define-property": {
@@ -1399,7 +1394,7 @@
           "integrity": "sha1-dp66rz9KY6rTr56NMEybvnm/sOY=",
           "dev": true,
           "requires": {
-            "is-descriptor": "^1.0.0"
+            "is-descriptor": "1.0.2"
           }
         },
         "is-accessor-descriptor": {
@@ -1408,7 +1403,7 @@
           "integrity": "sha512-m5hnHTkcVsPfqx3AKlyttIPb7J+XykHvJP2B9bZDjlhLIoEq4XoK64Vg7boZlVWYK6LUY94dYPEE7Lh0ZkZKcQ==",
           "dev": true,
           "requires": {
-            "kind-of": "^6.0.0"
+            "kind-of": "6.0.2"
           }
         },
         "is-data-descriptor": {
@@ -1417,7 +1412,7 @@
           "integrity": "sha512-jbRXy1FmtAoCjQkVmIVYwuuqDFUbaOeDjmed1tOGPrsMhtJA4rD9tkgA0F1qJ3gRFRXcHYVkdeaP50Q5rE/jLQ==",
           "dev": true,
           "requires": {
-            "kind-of": "^6.0.0"
+            "kind-of": "6.0.2"
           }
         },
         "is-descriptor": {
@@ -1426,9 +1421,9 @@
           "integrity": "sha512-2eis5WqQGV7peooDyLmNEPUrps9+SXX5c9pL3xEB+4e9HnGuDa7mB7kHxHw4CbqS9k1T2hOH3miL8n8WtiYVtg==",
           "dev": true,
           "requires": {
-            "is-accessor-descriptor": "^1.0.0",
-            "is-data-descriptor": "^1.0.0",
-            "kind-of": "^6.0.2"
+            "is-accessor-descriptor": "1.0.0",
+            "is-data-descriptor": "1.0.0",
+            "kind-of": "6.0.2"
           }
         },
         "isobject": {
@@ -1445,24 +1440,12 @@
         }
       }
     },
-    "base64-arraybuffer": {
-      "version": "0.1.2",
-      "resolved": "https://registry.npmjs.org/base64-arraybuffer/-/base64-arraybuffer-0.1.2.tgz",
-      "integrity": "sha1-R030qfLaJOBd8xWMOx2zw81GoVQ=",
-      "dev": true
-    },
     "base64-js": {
       "version": "1.3.0",
       "resolved": "https://registry.npmjs.org/base64-js/-/base64-js-1.3.0.tgz",
       "integrity": "sha512-ccav/yGvoa80BQDljCxsmmQ3Xvx60/UpBIij5QN21W3wBi/hhIC9OoO+KLpu9IJTS9j4DRVJ3aDDF9cMSoa2lw==",
       "dev": true
     },
-    "base64id": {
-      "version": "0.1.0",
-      "resolved": "https://registry.npmjs.org/base64id/-/base64id-0.1.0.tgz",
-      "integrity": "sha1-As4P3u4M709ACA4ec+g08LG/zj8=",
-      "dev": true
-    },
     "batch": {
       "version": "0.6.1",
       "resolved": "https://registry.npmjs.org/batch/-/batch-0.6.1.tgz",
@@ -1475,15 +1458,9 @@
       "integrity": "sha1-pDAdOJtqQ/m2f/PKEaP2Y342Dp4=",
       "dev": true,
       "requires": {
-        "tweetnacl": "^0.14.3"
+        "tweetnacl": "0.14.5"
       }
     },
-    "benchmark": {
-      "version": "1.0.0",
-      "resolved": "https://registry.npmjs.org/benchmark/-/benchmark-1.0.0.tgz",
-      "integrity": "sha1-Lx4vpMNZ8REiqhgwgiGOlX45DHM=",
-      "dev": true
-    },
     "better-assert": {
       "version": "1.0.2",
       "resolved": "https://registry.npmjs.org/better-assert/-/better-assert-1.0.2.tgz",
@@ -1505,19 +1482,13 @@
       "integrity": "sha512-DYWGk01lDcxeS/K9IHPGWfT8PsJmbXRtRd2Sx72Tnb8pcYZQFF1oSDb8hJtS1vhp212q1Rzi5dUf9+nq0o9UIg==",
       "dev": true
     },
-    "blob": {
-      "version": "0.0.4",
-      "resolved": "https://registry.npmjs.org/blob/-/blob-0.0.4.tgz",
-      "integrity": "sha1-vPEwUspURj8w+fx+lbmkdjCpSSE=",
-      "dev": true
-    },
     "blocking-proxy": {
       "version": "1.0.1",
       "resolved": "https://registry.npmjs.org/blocking-proxy/-/blocking-proxy-1.0.1.tgz",
       "integrity": "sha512-KE8NFMZr3mN2E0HcvCgRtX7DjhiIQrwle+nSVJVC/yqFb9+xznHl2ZcoBp2L9qzkI4t4cBFJ1efXF8Dwi132RA==",
       "dev": true,
       "requires": {
-        "minimist": "^1.2.0"
+        "minimist": "1.2.0"
       },
       "dependencies": {
         "minimist": {
@@ -1547,15 +1518,15 @@
       "dev": true,
       "requires": {
         "bytes": "3.0.0",
-        "content-type": "~1.0.4",
+        "content-type": "1.0.4",
         "debug": "2.6.9",
-        "depd": "~1.1.2",
-        "http-errors": "~1.6.3",
+        "depd": "1.1.2",
+        "http-errors": "1.6.3",
         "iconv-lite": "0.4.23",
-        "on-finished": "~2.3.0",
+        "on-finished": "2.3.0",
         "qs": "6.5.2",
         "raw-body": "2.3.3",
-        "type-is": "~1.6.16"
+        "type-is": "1.6.16"
       }
     },
     "bonjour": {
@@ -1564,12 +1535,12 @@
       "integrity": "sha1-jokKGD2O6aI5OzhExpGkK897yfU=",
       "dev": true,
       "requires": {
-        "array-flatten": "^2.1.0",
-        "deep-equal": "^1.0.1",
-        "dns-equal": "^1.0.0",
-        "dns-txt": "^2.0.2",
-        "multicast-dns": "^6.0.1",
-        "multicast-dns-service-types": "^1.1.0"
+        "array-flatten": "2.1.1",
+        "deep-equal": "1.0.1",
+        "dns-equal": "1.0.0",
+        "dns-txt": "2.0.2",
+        "multicast-dns": "6.2.3",
+        "multicast-dns-service-types": "1.1.0"
       },
       "dependencies": {
         "array-flatten": {
@@ -1603,9 +1574,9 @@
       "integrity": "sha1-mPxbQah4cO+cu5KXY1z4H1UF/bE=",
       "dev": true,
       "requires": {
-        "graceful-fs": "~2.0.0",
-        "mout": "~0.9.0",
-        "optimist": "~0.6.0",
+        "graceful-fs": "2.0.3",
+        "mout": "0.9.1",
+        "optimist": "0.6.1",
         "osenv": "0.0.3"
       },
       "dependencies": {
@@ -1623,7 +1594,7 @@
       "integrity": "sha512-iCuPHDFgrHX7H2vEI/5xpz07zSHB00TpugqhmYtVmMO6518mCuRMoOYFldEBl0g187ufozdaHgWKcYFb61qGiA==",
       "dev": true,
       "requires": {
-        "balanced-match": "^1.0.0",
+        "balanced-match": "1.0.0",
         "concat-map": "0.0.1"
       }
     },
@@ -1632,10 +1603,11 @@
       "resolved": "https://registry.npmjs.org/braces/-/braces-1.8.5.tgz",
       "integrity": "sha1-uneWLhLf+WnWt2cR6RS3N4V79qc=",
       "dev": true,
+      "optional": true,
       "requires": {
-        "expand-range": "^1.8.1",
-        "preserve": "^0.2.0",
-        "repeat-element": "^1.1.2"
+        "expand-range": "1.8.2",
+        "preserve": "0.2.0",
+        "repeat-element": "1.1.3"
       }
     },
     "brorand": {
@@ -1650,12 +1622,12 @@
       "integrity": "sha512-+7CHXqGuspUn/Sl5aO7Ea0xWGAtETPXNSAjHo48JfLdPWcMng33Xe4znFvQweqc/uzk5zSOI3H52CYnjCfb5hA==",
       "dev": true,
       "requires": {
-        "buffer-xor": "^1.0.3",
-        "cipher-base": "^1.0.0",
-        "create-hash": "^1.1.0",
-        "evp_bytestokey": "^1.0.3",
-        "inherits": "^2.0.1",
-        "safe-buffer": "^5.0.1"
+        "buffer-xor": "1.0.3",
+        "cipher-base": "1.0.4",
+        "create-hash": "1.2.0",
+        "evp_bytestokey": "1.0.3",
+        "inherits": "2.0.3",
+        "safe-buffer": "5.1.2"
       }
     },
     "browserify-cipher": {
@@ -1664,9 +1636,9 @@
       "integrity": "sha512-sPhkz0ARKbf4rRQt2hTpAHqn47X3llLkUGn+xEJzLjwY8LRs2p0v7ljvI5EyoRO/mexrNunNECisZs+gw2zz1w==",
       "dev": true,
       "requires": {
-        "browserify-aes": "^1.0.4",
-        "browserify-des": "^1.0.0",
-        "evp_bytestokey": "^1.0.0"
+        "browserify-aes": "1.2.0",
+        "browserify-des": "1.0.2",
+        "evp_bytestokey": "1.0.3"
       }
     },
     "browserify-des": {
@@ -1675,10 +1647,10 @@
       "integrity": "sha512-BioO1xf3hFwz4kc6iBhI3ieDFompMhrMlnDFC4/0/vd5MokpuAc3R+LYbwTA9A5Yc9pq9UYPqffKpW2ObuwX5A==",
       "dev": true,
       "requires": {
-        "cipher-base": "^1.0.1",
-        "des.js": "^1.0.0",
-        "inherits": "^2.0.1",
-        "safe-buffer": "^5.1.2"
+        "cipher-base": "1.0.4",
+        "des.js": "1.0.0",
+        "inherits": "2.0.3",
+        "safe-buffer": "5.1.2"
       }
     },
     "browserify-rsa": {
@@ -1687,8 +1659,8 @@
       "integrity": "sha1-IeCr+vbyApzy+vsTNWenAdQTVSQ=",
       "dev": true,
       "requires": {
-        "bn.js": "^4.1.0",
-        "randombytes": "^2.0.1"
+        "bn.js": "4.11.8",
+        "randombytes": "2.0.6"
       }
     },
     "browserify-sign": {
@@ -1697,13 +1669,13 @@
       "integrity": "sha1-qk62jl17ZYuqa/alfmMMvXqT0pg=",
       "dev": true,
       "requires": {
-        "bn.js": "^4.1.1",
-        "browserify-rsa": "^4.0.0",
-        "create-hash": "^1.1.0",
-        "create-hmac": "^1.1.2",
-        "elliptic": "^6.0.0",
-        "inherits": "^2.0.1",
-        "parse-asn1": "^5.0.0"
+        "bn.js": "4.11.8",
+        "browserify-rsa": "4.0.1",
+        "create-hash": "1.2.0",
+        "create-hmac": "1.1.7",
+        "elliptic": "6.4.1",
+        "inherits": "2.0.3",
+        "parse-asn1": "5.1.1"
       }
     },
     "browserify-zlib": {
@@ -1712,7 +1684,7 @@
       "integrity": "sha512-Z942RysHXmJrhqk88FmKBVq/v5tqmSkDz7p54G/MGyjMnCFFnC79XWNbg+Vta8W6Wb2qtSZTSxIGkJrRpCFEiA==",
       "dev": true,
       "requires": {
-        "pako": "~1.0.5"
+        "pako": "1.0.6"
       }
     },
     "browserslist": {
@@ -1721,8 +1693,8 @@
       "integrity": "sha1-C9dnBCWL6CmyOYu1Dkti0aFmsLk=",
       "dev": true,
       "requires": {
-        "caniuse-db": "^1.0.30000639",
-        "electron-to-chromium": "^1.2.7"
+        "caniuse-db": "1.0.30000906",
+        "electron-to-chromium": "1.3.83"
       }
     },
     "browserstack": {
@@ -1731,7 +1703,7 @@
       "integrity": "sha512-O8VMT64P9NOLhuIoD4YngyxBURefaSdR4QdhG8l6HZ9VxtU7jc3m6jLufFwKA5gaf7fetfB2TnRJnMxyob+heg==",
       "dev": true,
       "requires": {
-        "https-proxy-agent": "^2.2.1"
+        "https-proxy-agent": "2.2.1"
       }
     },
     "buffer": {
@@ -1740,9 +1712,9 @@
       "integrity": "sha1-bRu2AbB6TvztlwlBMgkwJ8lbwpg=",
       "dev": true,
       "requires": {
-        "base64-js": "^1.0.2",
-        "ieee754": "^1.1.4",
-        "isarray": "^1.0.0"
+        "base64-js": "1.3.0",
+        "ieee754": "1.1.12",
+        "isarray": "1.0.0"
       },
       "dependencies": {
         "isarray": {
@@ -1759,8 +1731,8 @@
       "integrity": "sha512-CFsHQgjtW1UChdXgbyJGtnm+O/uLQeZdtbDo8mfUgYXCHSM1wgrVxXm6bSyrUuErEb+4sYVGCzASBRot7zyrow==",
       "dev": true,
       "requires": {
-        "buffer-alloc-unsafe": "^1.1.0",
-        "buffer-fill": "^1.0.0"
+        "buffer-alloc-unsafe": "1.1.0",
+        "buffer-fill": "1.0.0"
       }
     },
     "buffer-alloc-unsafe": {
@@ -1817,19 +1789,19 @@
       "integrity": "sha512-Dph0MzuH+rTQzGPNT9fAnrPmMmjKfST6trxJeK7NQuHRaVw24VzPRWTmg9MpcwOVQZO0E1FBICUlFeNaKPIfHA==",
       "dev": true,
       "requires": {
-        "bluebird": "^3.5.1",
-        "chownr": "^1.0.1",
-        "glob": "^7.1.2",
-        "graceful-fs": "^4.1.11",
-        "lru-cache": "^4.1.1",
-        "mississippi": "^2.0.0",
-        "mkdirp": "^0.5.1",
-        "move-concurrently": "^1.0.1",
-        "promise-inflight": "^1.0.1",
-        "rimraf": "^2.6.2",
-        "ssri": "^5.2.4",
-        "unique-filename": "^1.1.0",
-        "y18n": "^4.0.0"
+        "bluebird": "3.5.2",
+        "chownr": "1.1.1",
+        "glob": "7.1.3",
+        "graceful-fs": "4.1.15",
+        "lru-cache": "4.1.3",
+        "mississippi": "2.0.0",
+        "mkdirp": "0.5.1",
+        "move-concurrently": "1.0.1",
+        "promise-inflight": "1.0.1",
+        "rimraf": "2.6.2",
+        "ssri": "5.3.0",
+        "unique-filename": "1.1.1",
+        "y18n": "4.0.0"
       },
       "dependencies": {
         "glob": {
@@ -1838,12 +1810,12 @@
           "integrity": "sha512-vcfuiIxogLV4DlGBHIUOwI0IbrJ8HWPc4MU7HzviGeNho/UJDfi6B5p3sHeWIQ0KGIU0Jpxi5ZHxemQfLkkAwQ==",
           "dev": true,
           "requires": {
-            "fs.realpath": "^1.0.0",
-            "inflight": "^1.0.4",
-            "inherits": "2",
-            "minimatch": "^3.0.4",
-            "once": "^1.3.0",
-            "path-is-absolute": "^1.0.0"
+            "fs.realpath": "1.0.0",
+            "inflight": "1.0.6",
+            "inherits": "2.0.3",
+            "minimatch": "3.0.4",
+            "once": "1.4.0",
+            "path-is-absolute": "1.0.1"
           }
         },
         "lru-cache": {
@@ -1852,8 +1824,8 @@
           "integrity": "sha512-fFEhvcgzuIoJVUF8fYr5KR0YqxD238zgObTps31YdADwPPAp82a4M8TrckkWyx7ekNlf9aBcVn81cFwwXngrJA==",
           "dev": true,
           "requires": {
-            "pseudomap": "^1.0.2",
-            "yallist": "^2.1.2"
+            "pseudomap": "1.0.2",
+            "yallist": "2.1.2"
           }
         },
         "minimatch": {
@@ -1862,7 +1834,7 @@
           "integrity": "sha512-yJHVQEhyqPLUTgt9B83PXu6W3rx4MvvHvSUvToogpwoGDOUQ+yDrR0HRot+yOCdCO7u4hX3pWft6kWBBcqh0UA==",
           "dev": true,
           "requires": {
-            "brace-expansion": "^1.1.7"
+            "brace-expansion": "1.1.11"
           }
         }
       }
@@ -1873,15 +1845,15 @@
       "integrity": "sha512-AKcdTnFSWATd5/GCPRxr2ChwIJ85CeyrEyjRHlKxQ56d4XJMGym0uAiKn0xbLOGOl3+yRpOTi484dVCEc5AUzQ==",
       "dev": true,
       "requires": {
-        "collection-visit": "^1.0.0",
-        "component-emitter": "^1.2.1",
-        "get-value": "^2.0.6",
-        "has-value": "^1.0.0",
-        "isobject": "^3.0.1",
-        "set-value": "^2.0.0",
-        "to-object-path": "^0.3.0",
-        "union-value": "^1.0.0",
-        "unset-value": "^1.0.0"
+        "collection-visit": "1.0.0",
+        "component-emitter": "1.2.1",
+        "get-value": "2.0.6",
+        "has-value": "1.0.0",
+        "isobject": "3.0.1",
+        "set-value": "2.0.0",
+        "to-object-path": "0.3.0",
+        "union-value": "1.0.0",
+        "unset-value": "1.0.0"
       },
       "dependencies": {
         "isobject": {
@@ -1898,7 +1870,7 @@
       "integrity": "sha1-lAhe9jWB7NPaqSREqP6U6CV3dR8=",
       "dev": true,
       "requires": {
-        "callsites": "^0.2.0"
+        "callsites": "0.2.0"
       }
     },
     "callsite": {
@@ -1918,8 +1890,8 @@
       "resolved": "https://registry.npmjs.org/camel-case/-/camel-case-1.2.2.tgz",
       "integrity": "sha1-Gsp8TRlTWaLOmVV5NDPG5VQlEfI=",
       "requires": {
-        "sentence-case": "^1.1.1",
-        "upper-case": "^1.1.1"
+        "sentence-case": "1.1.3",
+        "upper-case": "1.1.3"
       }
     },
     "camelcase": {
@@ -1933,8 +1905,8 @@
       "integrity": "sha1-MIvur/3ygRkFHvodkyITyRuPkuc=",
       "dev": true,
       "requires": {
-        "camelcase": "^2.0.0",
-        "map-obj": "^1.0.0"
+        "camelcase": "2.1.1",
+        "map-obj": "1.0.1"
       },
       "dependencies": {
         "camelcase": {
@@ -1951,10 +1923,10 @@
       "integrity": "sha1-tTTnxzTE+B7F++isoq0kNUuWLGw=",
       "dev": true,
       "requires": {
-        "browserslist": "^1.3.6",
-        "caniuse-db": "^1.0.30000529",
-        "lodash.memoize": "^4.1.2",
-        "lodash.uniq": "^4.5.0"
+        "browserslist": "1.7.7",
+        "caniuse-db": "1.0.30000906",
+        "lodash.memoize": "4.1.2",
+        "lodash.uniq": "4.5.0"
       }
     },
     "caniuse-db": {
@@ -1981,11 +1953,11 @@
       "integrity": "sha1-qBFcVeSnAv5NFQq9OHKCKn4J/Jg=",
       "dev": true,
       "requires": {
-        "ansi-styles": "^2.2.1",
-        "escape-string-regexp": "^1.0.2",
-        "has-ansi": "^2.0.0",
-        "strip-ansi": "^3.0.0",
-        "supports-color": "^2.0.0"
+        "ansi-styles": "2.2.1",
+        "escape-string-regexp": "1.0.5",
+        "has-ansi": "2.0.0",
+        "strip-ansi": "3.0.1",
+        "supports-color": "2.0.0"
       },
       "dependencies": {
         "supports-color": {
@@ -2001,21 +1973,21 @@
       "resolved": "https://registry.npmjs.org/change-case/-/change-case-2.1.6.tgz",
       "integrity": "sha1-UUryBRMVimj+fwDf9MMy1sKY0vk=",
       "requires": {
-        "camel-case": "^1.0.0",
-        "constant-case": "^1.0.0",
-        "dot-case": "^1.0.0",
-        "is-lower-case": "^1.0.0",
-        "is-upper-case": "^1.0.0",
-        "lower-case": "^1.0.0",
-        "param-case": "^1.0.0",
-        "pascal-case": "^1.0.0",
-        "path-case": "^1.0.0",
-        "sentence-case": "^1.0.0",
-        "snake-case": "^1.0.0",
-        "swap-case": "^1.0.0",
-        "title-case": "^1.0.0",
-        "upper-case": "^1.0.0",
-        "upper-case-first": "^1.0.0"
+        "camel-case": "1.2.2",
+        "constant-case": "1.1.2",
+        "dot-case": "1.1.2",
+        "is-lower-case": "1.1.3",
+        "is-upper-case": "1.1.2",
+        "lower-case": "1.1.4",
+        "param-case": "1.1.2",
+        "pascal-case": "1.1.2",
+        "path-case": "1.1.2",
+        "sentence-case": "1.1.3",
+        "snake-case": "1.1.2",
+        "swap-case": "1.1.2",
+        "title-case": "1.1.2",
+        "upper-case": "1.1.3",
+        "upper-case-first": "1.1.2"
       }
     },
     "cheerio": {
@@ -2023,10 +1995,10 @@
       "resolved": "https://registry.npmjs.org/cheerio/-/cheerio-0.12.4.tgz",
       "integrity": "sha1-wZlibp4esNQjOpGkeT5/iqppoYs=",
       "requires": {
-        "cheerio-select": "*",
-        "entities": "0.x",
+        "cheerio-select": "0.0.3",
+        "entities": "0.5.0",
         "htmlparser2": "3.1.4",
-        "underscore": "~1.4"
+        "underscore": "1.4.4"
       }
     },
     "cheerio-select": {
@@ -2034,7 +2006,7 @@
       "resolved": "https://registry.npmjs.org/cheerio-select/-/cheerio-select-0.0.3.tgz",
       "integrity": "sha1-PyQgEU88ywsbB1wkXM+q5dYXo4g=",
       "requires": {
-        "CSSselect": "0.x"
+        "CSSselect": "0.7.0"
       }
     },
     "chokidar": {
@@ -2042,16 +2014,16 @@
       "resolved": "https://registry.npmjs.org/chokidar/-/chokidar-1.7.0.tgz",
       "integrity": "sha1-eY5ol3gVHIB2tLNg5e3SjNortGg=",
       "dev": true,
+      "optional": true,
       "requires": {
-        "anymatch": "^1.3.0",
-        "async-each": "^1.0.0",
-        "fsevents": "^1.0.0",
-        "glob-parent": "^2.0.0",
-        "inherits": "^2.0.1",
-        "is-binary-path": "^1.0.0",
-        "is-glob": "^2.0.0",
-        "path-is-absolute": "^1.0.0",
-        "readdirp": "^2.0.0"
+        "anymatch": "1.3.2",
+        "async-each": "1.0.1",
+        "glob-parent": "2.0.0",
+        "inherits": "2.0.3",
+        "is-binary-path": "1.0.1",
+        "is-glob": "2.0.1",
+        "path-is-absolute": "1.0.1",
+        "readdirp": "2.2.1"
       }
     },
     "chownr": {
@@ -2066,7 +2038,7 @@
       "integrity": "sha512-xDbVgyfDTT2piup/h8dK/y4QZfJRSa73bw1WZ8b4XM1o7fsFubUVGYcE+1ANtOzJJELGpYoG2961z0Z6OAld9A==",
       "dev": true,
       "requires": {
-        "tslib": "^1.9.0"
+        "tslib": "1.9.3"
       }
     },
     "cipher-base": {
@@ -2075,8 +2047,8 @@
       "integrity": "sha512-Kkht5ye6ZGmwv40uUDZztayT2ThLQGfnj/T71N/XzeZeo3nf8foyW7zGTsPYkEya3m5f3cAypH+qe7YOrM1U2Q==",
       "dev": true,
       "requires": {
-        "inherits": "^2.0.1",
-        "safe-buffer": "^5.0.1"
+        "inherits": "2.0.3",
+        "safe-buffer": "5.1.2"
       }
     },
     "circular-json": {
@@ -2091,7 +2063,7 @@
       "integrity": "sha512-4CoL/A3hf90V3VIEjeuhSvlGFEHKzOz+Wfc2IVZc+FaUgU0ZQafJTP49fvnULipOPcAfqhyI2duwQyns6xqjYA==",
       "dev": true,
       "requires": {
-        "chalk": "^1.1.3"
+        "chalk": "1.1.3"
       }
     },
     "class-utils": {
@@ -2100,10 +2072,10 @@
       "integrity": "sha512-qOhPa/Fj7s6TY8H8esGu5QNpMMQxz79h+urzrNYN6mn+9BnxlDGf5QZ+XeCDsxSjPqsSR56XOZOJmpeurnLMeg==",
       "dev": true,
       "requires": {
-        "arr-union": "^3.1.0",
-        "define-property": "^0.2.5",
-        "isobject": "^3.0.0",
-        "static-extend": "^0.1.1"
+        "arr-union": "3.1.0",
+        "define-property": "0.2.5",
+        "isobject": "3.0.1",
+        "static-extend": "0.1.2"
       },
       "dependencies": {
         "define-property": {
@@ -2112,7 +2084,7 @@
           "integrity": "sha1-w1se+RjsPJkPmlvFe+BKrOxcgRY=",
           "dev": true,
           "requires": {
-            "is-descriptor": "^0.1.0"
+            "is-descriptor": "0.1.6"
           }
         },
         "isobject": {
@@ -2128,7 +2100,7 @@
       "resolved": "https://registry.npmjs.org/clean-css/-/clean-css-2.2.23.tgz",
       "integrity": "sha1-BZC1R4tRbEkD7cLYm9P9vdKGMow=",
       "requires": {
-        "commander": "2.2.x"
+        "commander": "2.2.0"
       }
     },
     "cli": {
@@ -2137,7 +2109,7 @@
       "integrity": "sha1-Aq1Eo4Cr8nraxebwzdewQ9dMU+M=",
       "requires": {
         "exit": "0.1.2",
-        "glob": "~ 3.2.1"
+        "glob": "3.2.11"
       }
     },
     "cli-cursor": {
@@ -2146,7 +2118,7 @@
       "integrity": "sha1-ZNo/fValRBLll5S9Ytw1KV6PKYc=",
       "dev": true,
       "requires": {
-        "restore-cursor": "^1.0.1"
+        "restore-cursor": "1.0.1"
       }
     },
     "cli-width": {
@@ -2161,9 +2133,9 @@
       "integrity": "sha512-4FG+RSG9DL7uEwRUZXZn3SS34DiDPfzP0VOiEwtUWlE+AR2EIg+hSyvrIgUUfhdgR/UkAeW2QHgeP+hWrXs7jQ==",
       "dev": true,
       "requires": {
-        "string-width": "^2.1.1",
-        "strip-ansi": "^4.0.0",
-        "wrap-ansi": "^2.0.0"
+        "string-width": "2.1.1",
+        "strip-ansi": "4.0.0",
+        "wrap-ansi": "2.1.0"
       },
       "dependencies": {
         "ansi-regex": {
@@ -2184,8 +2156,8 @@
           "integrity": "sha512-nOqH59deCq9SRHlxq1Aw85Jnt4w6KvLKqWVik6oA9ZklXLNIOlqg4F2yrT1MVaTjAqvVwdfeZ7w7aCvJD7ugkw==",
           "dev": true,
           "requires": {
-            "is-fullwidth-code-point": "^2.0.0",
-            "strip-ansi": "^4.0.0"
+            "is-fullwidth-code-point": "2.0.0",
+            "strip-ansi": "4.0.0"
           }
         },
         "strip-ansi": {
@@ -2194,7 +2166,7 @@
           "integrity": "sha1-qEeQIusaw2iocTibY1JixQXuNo8=",
           "dev": true,
           "requires": {
-            "ansi-regex": "^3.0.0"
+            "ansi-regex": "3.0.0"
           }
         }
       }
@@ -2217,7 +2189,7 @@
       "integrity": "sha1-qe8VNmDWqGqL3sAomlxoTSF0Mv0=",
       "dev": true,
       "requires": {
-        "q": "^1.1.2"
+        "q": "1.5.1"
       }
     },
     "code-point-at": {
@@ -2238,8 +2210,8 @@
       "integrity": "sha1-S8A3PBZLwykbTTaMgpzxqApZ3KA=",
       "dev": true,
       "requires": {
-        "map-visit": "^1.0.0",
-        "object-visit": "^1.0.0"
+        "map-visit": "1.0.0",
+        "object-visit": "1.0.1"
       }
     },
     "color": {
@@ -2248,9 +2220,9 @@
       "integrity": "sha1-bXtcdPtl6EHNSHkq0e1eB7kE12Q=",
       "dev": true,
       "requires": {
-        "clone": "^1.0.2",
-        "color-convert": "^1.3.0",
-        "color-string": "^0.3.0"
+        "clone": "1.0.4",
+        "color-convert": "1.9.3",
+        "color-string": "0.3.0"
       }
     },
     "color-convert": {
@@ -2274,7 +2246,7 @@
       "integrity": "sha1-J9RvtnAlxcL6JZk7+/V55HhBuZE=",
       "dev": true,
       "requires": {
-        "color-name": "^1.0.0"
+        "color-name": "1.1.3"
       }
     },
     "colormin": {
@@ -2283,9 +2255,9 @@
       "integrity": "sha1-6i90IKcrlogaOKrlnsEkpvcpgTM=",
       "dev": true,
       "requires": {
-        "color": "^0.11.0",
+        "color": "0.11.4",
         "css-color-names": "0.0.4",
-        "has": "^1.0.1"
+        "has": "1.0.3"
       }
     },
     "colors": {
@@ -2300,7 +2272,7 @@
       "integrity": "sha1-RYwH4J4NkA/Ci3Cj/sLazR0st/Y=",
       "dev": true,
       "requires": {
-        "lodash": "^4.5.0"
+        "lodash": "4.17.11"
       }
     },
     "combined-stream": {
@@ -2309,7 +2281,7 @@
       "integrity": "sha512-brWl9y6vOB1xYPZcpZde3N9zDByXTosAeMDo4p1wzo6UMOX4vumB+TP1RZ76sfE6Md68Q0NJSrE/gbezd4Ul+w==",
       "dev": true,
       "requires": {
-        "delayed-stream": "~1.0.0"
+        "delayed-stream": "1.0.0"
       }
     },
     "commander": {
@@ -2347,7 +2319,7 @@
       "integrity": "sha512-4aE67DL33dSW9gw4CI2H/yTxqHLNcxp0yS6jB+4h+wr3e43+1z7vm0HU9qXOH8j+qjKuL8+UtkOxYQSMq60Ylw==",
       "dev": true,
       "requires": {
-        "mime-db": ">= 1.36.0 < 2"
+        "mime-db": "1.37.0"
       }
     },
     "compression": {
@@ -2356,13 +2328,13 @@
       "integrity": "sha512-HSjyBG5N1Nnz7tF2+O7A9XUhyjru71/fwgNb7oIsEVHR0WShfs2tIS/EySLgiTe98aOK18YDlMXpzjCXY/n9mg==",
       "dev": true,
       "requires": {
-        "accepts": "~1.3.5",
+        "accepts": "1.3.5",
         "bytes": "3.0.0",
-        "compressible": "~2.0.14",
+        "compressible": "2.0.15",
         "debug": "2.6.9",
-        "on-headers": "~1.0.1",
+        "on-headers": "1.0.1",
         "safe-buffer": "5.1.2",
-        "vary": "~1.1.2"
+        "vary": "1.1.2"
       }
     },
     "concat-map": {
@@ -2377,10 +2349,10 @@
       "integrity": "sha512-27HBghJxjiZtIk3Ycvn/4kbJk/1uZuJFfuPEns6LaEvpvG1f0hTea8lilrouyo9mVc2GWdcEZ8OLoGmSADlrCw==",
       "dev": true,
       "requires": {
-        "buffer-from": "^1.0.0",
-        "inherits": "^2.0.3",
-        "readable-stream": "^2.2.2",
-        "typedarray": "^0.0.6"
+        "buffer-from": "1.1.1",
+        "inherits": "2.0.3",
+        "readable-stream": "2.3.6",
+        "typedarray": "0.0.6"
       },
       "dependencies": {
         "isarray": {
@@ -2395,13 +2367,13 @@
           "integrity": "sha512-tQtKA9WIAhBF3+VLAseyMqZeBjW0AHJoxOtYqSUZNJxauErmLbVm2FW1y+J/YA9dUrAC39ITejlZWhVIwawkKw==",
           "dev": true,
           "requires": {
-            "core-util-is": "~1.0.0",
-            "inherits": "~2.0.3",
-            "isarray": "~1.0.0",
-            "process-nextick-args": "~2.0.0",
-            "safe-buffer": "~5.1.1",
-            "string_decoder": "~1.1.1",
-            "util-deprecate": "~1.0.1"
+            "core-util-is": "1.0.2",
+            "inherits": "2.0.3",
+            "isarray": "1.0.0",
+            "process-nextick-args": "2.0.0",
+            "safe-buffer": "5.1.2",
+            "string_decoder": "1.1.1",
+            "util-deprecate": "1.0.2"
           }
         },
         "string_decoder": {
@@ -2410,7 +2382,7 @@
           "integrity": "sha512-n/ShnvDi6FHbbVfviro+WojiFzv+s8MPMHBczVePfUpDJLwoLT0ht1l4YwBCbi8pJAveEEdnkHyPyTP/mzRfwg==",
           "dev": true,
           "requires": {
-            "safe-buffer": "~5.1.0"
+            "safe-buffer": "5.1.2"
           }
         }
       }
@@ -2423,7 +2395,7 @@
       "requires": {
         "debug": "2.6.9",
         "finalhandler": "1.1.0",
-        "parseurl": "~1.3.2",
+        "parseurl": "1.3.2",
         "utils-merge": "1.0.1"
       },
       "dependencies": {
@@ -2434,12 +2406,12 @@
           "dev": true,
           "requires": {
             "debug": "2.6.9",
-            "encodeurl": "~1.0.1",
-            "escape-html": "~1.0.3",
-            "on-finished": "~2.3.0",
-            "parseurl": "~1.3.2",
-            "statuses": "~1.3.1",
-            "unpipe": "~1.0.0"
+            "encodeurl": "1.0.2",
+            "escape-html": "1.0.3",
+            "on-finished": "2.3.0",
+            "parseurl": "1.3.2",
+            "statuses": "1.3.1",
+            "unpipe": "1.0.0"
           }
         },
         "statuses": {
@@ -2462,7 +2434,7 @@
       "integrity": "sha1-8CQcRXMKn8YyOyBtvzjtx0HQuxA=",
       "dev": true,
       "requires": {
-        "date-now": "^0.1.4"
+        "date-now": "0.1.4"
       }
     },
     "constant-case": {
@@ -2470,8 +2442,8 @@
       "resolved": "https://registry.npmjs.org/constant-case/-/constant-case-1.1.2.tgz",
       "integrity": "sha1-jsLKW6ND4Aqjjb9OIA/VrJB+/WM=",
       "requires": {
-        "snake-case": "^1.1.0",
-        "upper-case": "^1.1.1"
+        "snake-case": "1.1.2",
+        "upper-case": "1.1.3"
       }
     },
     "constants-browserify": {
@@ -2504,7 +2476,7 @@
       "integrity": "sha512-eFu7XigvxdZ1ETfbgPBohgyQ/Z++C0eEhTor0qRwBw9unw+L0/6V8wkSuGgzdThkiS5lSpdptOQPD8Ak40a+7A==",
       "dev": true,
       "requires": {
-        "safe-buffer": "~5.1.1"
+        "safe-buffer": "5.1.2"
       }
     },
     "cookie": {
@@ -2525,12 +2497,12 @@
       "integrity": "sha512-f2domd9fsVDFtaFcbaRZuYXwtdmnzqbADSwhSWYxYB/Q8zsdUUFMXVRwXGDMWmbEzAn1kdRrtI1T/KTFOL4X2A==",
       "dev": true,
       "requires": {
-        "aproba": "^1.1.1",
-        "fs-write-stream-atomic": "^1.0.8",
-        "iferr": "^0.1.5",
-        "mkdirp": "^0.5.1",
-        "rimraf": "^2.5.4",
-        "run-queue": "^1.0.0"
+        "aproba": "1.2.0",
+        "fs-write-stream-atomic": "1.0.10",
+        "iferr": "0.1.5",
+        "mkdirp": "0.5.1",
+        "rimraf": "2.6.2",
+        "run-queue": "1.0.3"
       }
     },
     "copy-descriptor": {
@@ -2545,14 +2517,14 @@
       "integrity": "sha512-Y+SQCF+0NoWQryez2zXn5J5knmr9z/9qSQt7fbL78u83rxmigOy8X5+BFn8CFSuX+nKT8gpYwJX68ekqtQt6ZA==",
       "dev": true,
       "requires": {
-        "cacache": "^10.0.4",
-        "find-cache-dir": "^1.0.0",
-        "globby": "^7.1.1",
-        "is-glob": "^4.0.0",
-        "loader-utils": "^1.1.0",
-        "minimatch": "^3.0.4",
-        "p-limit": "^1.0.0",
-        "serialize-javascript": "^1.4.0"
+        "cacache": "10.0.4",
+        "find-cache-dir": "1.0.0",
+        "globby": "7.1.1",
+        "is-glob": "4.0.0",
+        "loader-utils": "1.1.0",
+        "minimatch": "3.0.4",
+        "p-limit": "1.3.0",
+        "serialize-javascript": "1.5.0"
       },
       "dependencies": {
         "is-extglob": {
@@ -2567,7 +2539,7 @@
           "integrity": "sha1-lSHHaEXMJhCoUgPd8ICpWML/q8A=",
           "dev": true,
           "requires": {
-            "is-extglob": "^2.1.1"
+            "is-extglob": "2.1.1"
           }
         },
         "minimatch": {
@@ -2576,7 +2548,7 @@
           "integrity": "sha512-yJHVQEhyqPLUTgt9B83PXu6W3rx4MvvHvSUvToogpwoGDOUQ+yDrR0HRot+yOCdCO7u4hX3pWft6kWBBcqh0UA==",
           "dev": true,
           "requires": {
-            "brace-expansion": "^1.1.7"
+            "brace-expansion": "1.1.11"
           }
         }
       }
@@ -2598,10 +2570,10 @@
       "integrity": "sha512-6e5vDdrXZD+t5v0L8CrurPeybg4Fmf+FCSYxXKYVAqLUtyCSbuyqE059d0kDthTNRzKVjL7QMgNpEUlsoYH3iQ==",
       "dev": true,
       "requires": {
-        "is-directory": "^0.3.1",
-        "js-yaml": "^3.9.0",
-        "parse-json": "^4.0.0",
-        "require-from-string": "^2.0.1"
+        "is-directory": "0.3.1",
+        "js-yaml": "3.12.0",
+        "parse-json": "4.0.0",
+        "require-from-string": "2.0.2"
       },
       "dependencies": {
         "esprima": {
@@ -2616,8 +2588,8 @@
           "integrity": "sha512-PIt2cnwmPfL4hKNwqeiuz4bKfnzHTBv6HyVgjahA6mPLwPDzjDWrplJBMjHUFxku/N3FlmrbyPclad+I+4mJ3A==",
           "dev": true,
           "requires": {
-            "argparse": "^1.0.7",
-            "esprima": "^4.0.0"
+            "argparse": "1.0.10",
+            "esprima": "4.0.1"
           }
         },
         "parse-json": {
@@ -2626,8 +2598,8 @@
           "integrity": "sha1-vjX1Qlvh9/bHRxhPmKeIy5lHfuA=",
           "dev": true,
           "requires": {
-            "error-ex": "^1.3.1",
-            "json-parse-better-errors": "^1.0.1"
+            "error-ex": "1.3.2",
+            "json-parse-better-errors": "1.0.2"
           }
         }
       }
@@ -2638,8 +2610,8 @@
       "integrity": "sha512-GbEHQPMOswGpKXM9kCWVrremUcBmjteUaQ01T9rkKCPDXfUHX0IoP9LpHYo2NPFampa4e+/pFDc3jQdxrxQLaw==",
       "dev": true,
       "requires": {
-        "bn.js": "^4.1.0",
-        "elliptic": "^6.0.0"
+        "bn.js": "4.11.8",
+        "elliptic": "6.4.1"
       }
     },
     "create-hash": {
@@ -2648,11 +2620,11 @@
       "integrity": "sha512-z00bCGNHDG8mHAkP7CtT1qVu+bFQUPjYq/4Iv3C3kWjTFV10zIjfSoeqXo9Asws8gwSHDGj/hl2u4OGIjapeCg==",
       "dev": true,
       "requires": {
-        "cipher-base": "^1.0.1",
-        "inherits": "^2.0.1",
-        "md5.js": "^1.3.4",
-        "ripemd160": "^2.0.1",
-        "sha.js": "^2.4.0"
+        "cipher-base": "1.0.4",
+        "inherits": "2.0.3",
+        "md5.js": "1.3.5",
+        "ripemd160": "2.0.2",
+        "sha.js": "2.4.11"
       }
     },
     "create-hmac": {
@@ -2661,12 +2633,12 @@
       "integrity": "sha512-MJG9liiZ+ogc4TzUwuvbER1JRdgvUFSB5+VR/g5h82fGaIRWMWddtKBHi7/sVhfjQZ6SehlyhvQYrcYkaUIpLg==",
       "dev": true,
       "requires": {
-        "cipher-base": "^1.0.3",
-        "create-hash": "^1.1.0",
-        "inherits": "^2.0.1",
-        "ripemd160": "^2.0.0",
-        "safe-buffer": "^5.0.1",
-        "sha.js": "^2.4.8"
+        "cipher-base": "1.0.4",
+        "create-hash": "1.2.0",
+        "inherits": "2.0.3",
+        "ripemd160": "2.0.2",
+        "safe-buffer": "5.1.2",
+        "sha.js": "2.4.11"
       }
     },
     "cross-spawn": {
@@ -2675,11 +2647,11 @@
       "integrity": "sha512-eTVLrBSt7fjbDygz805pMnstIs2VTBNkRm0qxZd+M7A5XDdxVRWO5MxGBXZhjY4cqLYLdtrGqRf8mBPmzwSpWQ==",
       "dev": true,
       "requires": {
-        "nice-try": "^1.0.4",
-        "path-key": "^2.0.1",
-        "semver": "^5.5.0",
-        "shebang-command": "^1.2.0",
-        "which": "^1.2.9"
+        "nice-try": "1.0.5",
+        "path-key": "2.0.1",
+        "semver": "5.6.0",
+        "shebang-command": "1.2.0",
+        "which": "1.3.1"
       },
       "dependencies": {
         "which": {
@@ -2688,7 +2660,7 @@
           "integrity": "sha512-HxJdYWq1MTIQbJ3nw0cqssHoTNU267KlrDuGZ1WYlxDStUtKUhOaJmh112/TZmHxxUfuJqPXSOm7tDyas0OSIQ==",
           "dev": true,
           "requires": {
-            "isexe": "^2.0.0"
+            "isexe": "2.0.0"
           }
         }
       }
@@ -2699,17 +2671,17 @@
       "integrity": "sha512-fz4spIh+znjO2VjL+IdhEpRJ3YN6sMzITSBijk6FK2UvTqruSQW+/cCZTSNsMiZNvUeq0CqurF+dAbyiGOY6Wg==",
       "dev": true,
       "requires": {
-        "browserify-cipher": "^1.0.0",
-        "browserify-sign": "^4.0.0",
-        "create-ecdh": "^4.0.0",
-        "create-hash": "^1.1.0",
-        "create-hmac": "^1.1.0",
-        "diffie-hellman": "^5.0.0",
-        "inherits": "^2.0.1",
-        "pbkdf2": "^3.0.3",
-        "public-encrypt": "^4.0.0",
-        "randombytes": "^2.0.0",
-        "randomfill": "^1.0.3"
+        "browserify-cipher": "1.0.1",
+        "browserify-sign": "4.0.4",
+        "create-ecdh": "4.0.3",
+        "create-hash": "1.2.0",
+        "create-hmac": "1.1.7",
+        "diffie-hellman": "5.0.3",
+        "inherits": "2.0.3",
+        "pbkdf2": "3.0.17",
+        "public-encrypt": "4.0.3",
+        "randombytes": "2.0.6",
+        "randomfill": "1.0.4"
       }
     },
     "cson-parser": {
@@ -2718,7 +2690,7 @@
       "integrity": "sha1-fsZ14DkUVTO/KmqFYHPxWZ2cLSQ=",
       "dev": true,
       "requires": {
-        "coffee-script": "^1.10.0"
+        "coffee-script": "1.12.7"
       },
       "dependencies": {
         "coffee-script": {
@@ -2741,18 +2713,18 @@
       "integrity": "sha1-th6eMNuUMD5v/IkvEOzQmtAlof0=",
       "dev": true,
       "requires": {
-        "babel-code-frame": "^6.11.0",
-        "css-selector-tokenizer": "^0.7.0",
-        "cssnano": ">=2.6.1 <4",
-        "loader-utils": "^1.0.2",
-        "lodash.camelcase": "^4.3.0",
-        "object-assign": "^4.0.1",
-        "postcss": "^5.0.6",
-        "postcss-modules-extract-imports": "^1.0.0",
-        "postcss-modules-local-by-default": "^1.0.1",
-        "postcss-modules-scope": "^1.0.0",
-        "postcss-modules-values": "^1.1.0",
-        "source-list-map": "^0.1.7"
+        "babel-code-frame": "6.26.0",
+        "css-selector-tokenizer": "0.7.1",
+        "cssnano": "3.10.0",
+        "loader-utils": "1.1.0",
+        "lodash.camelcase": "4.3.0",
+        "object-assign": "4.1.1",
+        "postcss": "5.2.18",
+        "postcss-modules-extract-imports": "1.2.1",
+        "postcss-modules-local-by-default": "1.2.0",
+        "postcss-modules-scope": "1.1.0",
+        "postcss-modules-values": "1.3.0",
+        "source-list-map": "0.1.8"
       }
     },
     "css-select": {
@@ -2761,10 +2733,10 @@
       "integrity": "sha1-KzoRBTnFNV8c2NMUYj6HCxIeyFg=",
       "dev": true,
       "requires": {
-        "boolbase": "~1.0.0",
-        "css-what": "2.1",
+        "boolbase": "1.0.0",
+        "css-what": "2.1.2",
         "domutils": "1.5.1",
-        "nth-check": "~1.0.1"
+        "nth-check": "1.0.2"
       },
       "dependencies": {
         "domutils": {
@@ -2773,8 +2745,8 @@
           "integrity": "sha1-3NhIiib1Y9YQeeSMn3t+Mjc2gs8=",
           "dev": true,
           "requires": {
-            "dom-serializer": "0",
-            "domelementtype": "1"
+            "dom-serializer": "0.1.0",
+            "domelementtype": "1.2.1"
           }
         }
       }
@@ -2785,9 +2757,9 @@
       "integrity": "sha512-xYL0AMZJ4gFzJQsHUKa5jiWWi2vH77WVNg7JYRyewwj6oPh4yb/y6Y9ZCw9dsj/9UauMhtuxR+ogQd//EdEVNA==",
       "dev": true,
       "requires": {
-        "cssesc": "^0.1.0",
-        "fastparse": "^1.1.1",
-        "regexpu-core": "^1.0.0"
+        "cssesc": "0.1.0",
+        "fastparse": "1.1.2",
+        "regexpu-core": "1.0.0"
       },
       "dependencies": {
         "regexpu-core": {
@@ -2796,9 +2768,9 @@
           "integrity": "sha1-hqdj9Y7k18L2sQLkdkBQ3n7ZDGs=",
           "dev": true,
           "requires": {
-            "regenerate": "^1.2.1",
-            "regjsgen": "^0.2.0",
-            "regjsparser": "^0.1.4"
+            "regenerate": "1.4.0",
+            "regjsgen": "0.2.0",
+            "regjsparser": "0.1.5"
           }
         }
       }
@@ -2821,7 +2793,7 @@
       "integrity": "sha1-OmoE51Zcjp0ZvrSXZ8fslug2WAU=",
       "dev": true,
       "requires": {
-        "parserlib": "~0.2.2"
+        "parserlib": "0.2.5"
       }
     },
     "cssnano": {
@@ -2830,38 +2802,38 @@
       "integrity": "sha1-Tzj2zqK5sX+gFJDyPx3GjqZcHDg=",
       "dev": true,
       "requires": {
-        "autoprefixer": "^6.3.1",
-        "decamelize": "^1.1.2",
-        "defined": "^1.0.0",
-        "has": "^1.0.1",
-        "object-assign": "^4.0.1",
-        "postcss": "^5.0.14",
-        "postcss-calc": "^5.2.0",
-        "postcss-colormin": "^2.1.8",
-        "postcss-convert-values": "^2.3.4",
-        "postcss-discard-comments": "^2.0.4",
-        "postcss-discard-duplicates": "^2.0.1",
-        "postcss-discard-empty": "^2.0.1",
-        "postcss-discard-overridden": "^0.1.1",
-        "postcss-discard-unused": "^2.2.1",
-        "postcss-filter-plugins": "^2.0.0",
-        "postcss-merge-idents": "^2.1.5",
-        "postcss-merge-longhand": "^2.0.1",
-        "postcss-merge-rules": "^2.0.3",
-        "postcss-minify-font-values": "^1.0.2",
-        "postcss-minify-gradients": "^1.0.1",
-        "postcss-minify-params": "^1.0.4",
-        "postcss-minify-selectors": "^2.0.4",
-        "postcss-normalize-charset": "^1.1.0",
-        "postcss-normalize-url": "^3.0.7",
-        "postcss-ordered-values": "^2.1.0",
-        "postcss-reduce-idents": "^2.2.2",
-        "postcss-reduce-initial": "^1.0.0",
-        "postcss-reduce-transforms": "^1.0.3",
-        "postcss-svgo": "^2.1.1",
-        "postcss-unique-selectors": "^2.0.2",
-        "postcss-value-parser": "^3.2.3",
-        "postcss-zindex": "^2.0.1"
+        "autoprefixer": "6.7.7",
+        "decamelize": "1.2.0",
+        "defined": "1.0.0",
+        "has": "1.0.3",
+        "object-assign": "4.1.1",
+        "postcss": "5.2.18",
+        "postcss-calc": "5.3.1",
+        "postcss-colormin": "2.2.2",
+        "postcss-convert-values": "2.6.1",
+        "postcss-discard-comments": "2.0.4",
+        "postcss-discard-duplicates": "2.1.0",
+        "postcss-discard-empty": "2.1.0",
+        "postcss-discard-overridden": "0.1.1",
+        "postcss-discard-unused": "2.2.3",
+        "postcss-filter-plugins": "2.0.3",
+        "postcss-merge-idents": "2.1.7",
+        "postcss-merge-longhand": "2.0.2",
+        "postcss-merge-rules": "2.1.2",
+        "postcss-minify-font-values": "1.0.5",
+        "postcss-minify-gradients": "1.0.5",
+        "postcss-minify-params": "1.2.2",
+        "postcss-minify-selectors": "2.1.1",
+        "postcss-normalize-charset": "1.1.1",
+        "postcss-normalize-url": "3.0.8",
+        "postcss-ordered-values": "2.2.3",
+        "postcss-reduce-idents": "2.4.0",
+        "postcss-reduce-initial": "1.0.1",
+        "postcss-reduce-transforms": "1.0.4",
+        "postcss-svgo": "2.1.6",
+        "postcss-unique-selectors": "2.0.2",
+        "postcss-value-parser": "3.3.1",
+        "postcss-zindex": "2.2.0"
       }
     },
     "csso": {
@@ -2870,8 +2842,8 @@
       "integrity": "sha1-3dUsWHAz9J6Utx/FVWnyUuj/X4U=",
       "dev": true,
       "requires": {
-        "clap": "^1.0.9",
-        "source-map": "^0.5.3"
+        "clap": "1.2.3",
+        "source-map": "0.5.7"
       },
       "dependencies": {
         "source-map": {
@@ -2888,7 +2860,7 @@
       "integrity": "sha1-mI3zP+qxke95mmE2nddsF635V+o=",
       "dev": true,
       "requires": {
-        "array-find-index": "^1.0.1"
+        "array-find-index": "1.0.2"
       }
     },
     "custom-event": {
@@ -2909,7 +2881,7 @@
       "integrity": "sha1-dUu1v+VUUdpppYuU1F9MWwRi1Y8=",
       "dev": true,
       "requires": {
-        "es5-ext": "^0.10.9"
+        "es5-ext": "0.10.46"
       }
     },
     "dashdash": {
@@ -2918,9 +2890,15 @@
       "integrity": "sha1-hTz6D3y+L+1d4gMmuN1YEDX24vA=",
       "dev": true,
       "requires": {
-        "assert-plus": "^1.0.0"
+        "assert-plus": "1.0.0"
       }
     },
+    "date-format": {
+      "version": "1.2.0",
+      "resolved": "https://registry.npmjs.org/date-format/-/date-format-1.2.0.tgz",
+      "integrity": "sha1-YV6CjiM90aubua4JUODOzPpuytg=",
+      "dev": true
+    },
     "date-now": {
       "version": "0.1.4",
       "resolved": "https://registry.npmjs.org/date-now/-/date-now-0.1.4.tgz",
@@ -2977,8 +2955,8 @@
       "integrity": "sha512-lAc4i9QJR0YHSDFdzeBQKfZ1SRDG3hsJNEkrpcZa8QhBfidLAilT60BDEIVUUGqosFp425KOgB3uYqcnQrWafQ==",
       "dev": true,
       "requires": {
-        "execa": "^0.10.0",
-        "ip-regex": "^2.1.0"
+        "execa": "0.10.0",
+        "ip-regex": "2.1.0"
       }
     },
     "define-properties": {
@@ -2987,7 +2965,7 @@
       "integrity": "sha512-3MqfYKj2lLzdMSf8ZIZE/V+Zuy+BgD6f164e8K2w7dgnpKArBDerGYpM46IYYcjnkdPNMjPk9A6VFB8+3SKlXQ==",
       "dev": true,
       "requires": {
-        "object-keys": "^1.0.12"
+        "object-keys": "1.0.12"
       }
     },
     "define-property": {
@@ -2996,8 +2974,8 @@
       "integrity": "sha512-jwK2UV4cnPpbcG7+VRARKTZPUWowwXA8bzH5NP6ud0oeAxyYPuGZUAC7hMugpCdz4BeSZl2Dl9k66CHJ/46ZYQ==",
       "dev": true,
       "requires": {
-        "is-descriptor": "^1.0.2",
-        "isobject": "^3.0.1"
+        "is-descriptor": "1.0.2",
+        "isobject": "3.0.1"
       },
       "dependencies": {
         "is-accessor-descriptor": {
@@ -3006,7 +2984,7 @@
           "integrity": "sha512-m5hnHTkcVsPfqx3AKlyttIPb7J+XykHvJP2B9bZDjlhLIoEq4XoK64Vg7boZlVWYK6LUY94dYPEE7Lh0ZkZKcQ==",
           "dev": true,
           "requires": {
-            "kind-of": "^6.0.0"
+            "kind-of": "6.0.2"
           }
         },
         "is-data-descriptor": {
@@ -3015,7 +2993,7 @@
           "integrity": "sha512-jbRXy1FmtAoCjQkVmIVYwuuqDFUbaOeDjmed1tOGPrsMhtJA4rD9tkgA0F1qJ3gRFRXcHYVkdeaP50Q5rE/jLQ==",
           "dev": true,
           "requires": {
-            "kind-of": "^6.0.0"
+            "kind-of": "6.0.2"
           }
         },
         "is-descriptor": {
@@ -3024,9 +3002,9 @@
           "integrity": "sha512-2eis5WqQGV7peooDyLmNEPUrps9+SXX5c9pL3xEB+4e9HnGuDa7mB7kHxHw4CbqS9k1T2hOH3miL8n8WtiYVtg==",
           "dev": true,
           "requires": {
-            "is-accessor-descriptor": "^1.0.0",
-            "is-data-descriptor": "^1.0.0",
-            "kind-of": "^6.0.2"
+            "is-accessor-descriptor": "1.0.0",
+            "is-data-descriptor": "1.0.0",
+            "kind-of": "6.0.2"
           }
         },
         "isobject": {
@@ -3055,13 +3033,13 @@
       "integrity": "sha1-wSyYHQZ4RshLyvhiz/kw2Qf/0ag=",
       "dev": true,
       "requires": {
-        "globby": "^5.0.0",
-        "is-path-cwd": "^1.0.0",
-        "is-path-in-cwd": "^1.0.0",
-        "object-assign": "^4.0.1",
-        "pify": "^2.0.0",
-        "pinkie-promise": "^2.0.0",
-        "rimraf": "^2.2.8"
+        "globby": "5.0.0",
+        "is-path-cwd": "1.0.0",
+        "is-path-in-cwd": "1.0.1",
+        "object-assign": "4.1.1",
+        "pify": "2.3.0",
+        "pinkie-promise": "2.0.1",
+        "rimraf": "2.6.2"
       },
       "dependencies": {
         "glob": {
@@ -3070,12 +3048,12 @@
           "integrity": "sha512-vcfuiIxogLV4DlGBHIUOwI0IbrJ8HWPc4MU7HzviGeNho/UJDfi6B5p3sHeWIQ0KGIU0Jpxi5ZHxemQfLkkAwQ==",
           "dev": true,
           "requires": {
-            "fs.realpath": "^1.0.0",
-            "inflight": "^1.0.4",
-            "inherits": "2",
-            "minimatch": "^3.0.4",
-            "once": "^1.3.0",
-            "path-is-absolute": "^1.0.0"
+            "fs.realpath": "1.0.0",
+            "inflight": "1.0.6",
+            "inherits": "2.0.3",
+            "minimatch": "3.0.4",
+            "once": "1.4.0",
+            "path-is-absolute": "1.0.1"
           }
         },
         "globby": {
@@ -3084,12 +3062,12 @@
           "integrity": "sha1-69hGZ8oNuzMLmbz8aOrCvFQ3Dg0=",
           "dev": true,
           "requires": {
-            "array-union": "^1.0.1",
-            "arrify": "^1.0.0",
-            "glob": "^7.0.3",
-            "object-assign": "^4.0.1",
-            "pify": "^2.0.0",
-            "pinkie-promise": "^2.0.0"
+            "array-union": "1.0.2",
+            "arrify": "1.0.1",
+            "glob": "7.1.3",
+            "object-assign": "4.1.1",
+            "pify": "2.3.0",
+            "pinkie-promise": "2.0.1"
           }
         },
         "minimatch": {
@@ -3098,7 +3076,7 @@
           "integrity": "sha512-yJHVQEhyqPLUTgt9B83PXu6W3rx4MvvHvSUvToogpwoGDOUQ+yDrR0HRot+yOCdCO7u4hX3pWft6kWBBcqh0UA==",
           "dev": true,
           "requires": {
-            "brace-expansion": "^1.1.7"
+            "brace-expansion": "1.1.11"
           }
         },
         "pify": {
@@ -3127,8 +3105,8 @@
       "integrity": "sha1-wHTS4qpqipoH29YfmhXCzYPsjsw=",
       "dev": true,
       "requires": {
-        "inherits": "^2.0.1",
-        "minimalistic-assert": "^1.0.0"
+        "inherits": "2.0.3",
+        "minimalistic-assert": "1.0.1"
       }
     },
     "destroy": {
@@ -3143,7 +3121,7 @@
       "integrity": "sha1-920GQ1LN9Docts5hnE7jqUdd4gg=",
       "dev": true,
       "requires": {
-        "repeating": "^2.0.0"
+        "repeating": "2.0.1"
       }
     },
     "detect-node": {
@@ -3175,9 +3153,9 @@
       "integrity": "sha512-kqag/Nl+f3GwyK25fhUMYj81BUOrZ9IuJsjIcDE5icNM9FJHAVm3VcUDxdLPoQtTuUylWm6ZIknYJwwaPxsUzg==",
       "dev": true,
       "requires": {
-        "bn.js": "^4.1.0",
-        "miller-rabin": "^4.0.0",
-        "randombytes": "^2.0.0"
+        "bn.js": "4.11.8",
+        "miller-rabin": "4.0.1",
+        "randombytes": "2.0.6"
       }
     },
     "dir-glob": {
@@ -3186,8 +3164,8 @@
       "integrity": "sha512-37qirFDz8cA5fimp9feo43fSuRo2gHwaIn6dXL8Ber1dGwUosDrGZeCCXq57WnIqE4aQ+u3eQZzsk1yOzhdwag==",
       "dev": true,
       "requires": {
-        "arrify": "^1.0.1",
-        "path-type": "^3.0.0"
+        "arrify": "1.0.1",
+        "path-type": "3.0.0"
       }
     },
     "dns-equal": {
@@ -3202,8 +3180,8 @@
       "integrity": "sha512-0UxfQkMhYAUaZI+xrNZOz/as5KgDU0M/fQ9b6SpkyLbk3GEswDi6PADJVaYJradtRVsRIlF1zLyOodbcTCDzUg==",
       "dev": true,
       "requires": {
-        "ip": "^1.1.0",
-        "safe-buffer": "^5.0.1"
+        "ip": "1.1.5",
+        "safe-buffer": "5.1.2"
       }
     },
     "dns-txt": {
@@ -3212,7 +3190,7 @@
       "integrity": "sha1-uR2Ab10nGI5Ks+fRB9iBocxGQrY=",
       "dev": true,
       "requires": {
-        "buffer-indexof": "^1.0.0"
+        "buffer-indexof": "1.1.1"
       }
     },
     "doctrine": {
@@ -3221,7 +3199,7 @@
       "integrity": "sha512-35mSku4ZXK0vfCuHEDAwt55dg2jNajHZ1odvF+8SSr82EsZY4QmXfuWso8oEd8zRhVObSN18aM0CjSdoBX7zIw==",
       "dev": true,
       "requires": {
-        "esutils": "^2.0.2"
+        "esutils": "2.0.2"
       }
     },
     "dom-converter": {
@@ -3230,7 +3208,7 @@
       "integrity": "sha512-gd3ypIPfOMr9h5jIKq8E3sHOTCjeirnl0WK5ZdS1AW0Odt0b1PaWaHdJ4Qk4klv+YB9aJBS7mESXjFoDQPu6DA==",
       "dev": true,
       "requires": {
-        "utila": "~0.4"
+        "utila": "0.4.0"
       }
     },
     "dom-serialize": {
@@ -3239,10 +3217,10 @@
       "integrity": "sha1-ViromZ9Evl6jB29UGdzVnrQ6yVs=",
       "dev": true,
       "requires": {
-        "custom-event": "~1.0.0",
-        "ent": "~2.2.0",
-        "extend": "^3.0.0",
-        "void-elements": "^2.0.0"
+        "custom-event": "1.0.1",
+        "ent": "2.2.0",
+        "extend": "3.0.2",
+        "void-elements": "2.0.1"
       }
     },
     "dom-serializer": {
@@ -3251,8 +3229,8 @@
       "integrity": "sha1-BzxpdUbOB4DOI75KKOKT5AvDDII=",
       "dev": true,
       "requires": {
-        "domelementtype": "~1.1.1",
-        "entities": "~1.1.1"
+        "domelementtype": "1.1.3",
+        "entities": "1.1.2"
       },
       "dependencies": {
         "domelementtype": {
@@ -3285,7 +3263,7 @@
       "resolved": "https://registry.npmjs.org/domhandler/-/domhandler-2.0.3.tgz",
       "integrity": "sha1-iJ+N9iZAOvB4jinWbV1cb36/D9Y=",
       "requires": {
-        "domelementtype": "1"
+        "domelementtype": "1.2.1"
       }
     },
     "domutils": {
@@ -3293,7 +3271,7 @@
       "resolved": "https://registry.npmjs.org/domutils/-/domutils-1.4.3.tgz",
       "integrity": "sha1-CGVRN5bGswYDGFDhdVFrr4C3Km8=",
       "requires": {
-        "domelementtype": "1"
+        "domelementtype": "1.2.1"
       }
     },
     "dot-case": {
@@ -3301,7 +3279,7 @@
       "resolved": "https://registry.npmjs.org/dot-case/-/dot-case-1.1.2.tgz",
       "integrity": "sha1-HnOCaQDeKNbeVIC8HeMdCEKwa+w=",
       "requires": {
-        "sentence-case": "^1.1.2"
+        "sentence-case": "1.1.3"
       }
     },
     "duplexify": {
@@ -3310,10 +3288,10 @@
       "integrity": "sha512-vM58DwdnKmty+FSPzT14K9JXb90H+j5emaR4KYbr2KTIz00WHGbWOe5ghQTx233ZCLZtrGDALzKwcjEtSt35mA==",
       "dev": true,
       "requires": {
-        "end-of-stream": "^1.0.0",
-        "inherits": "^2.0.1",
-        "readable-stream": "^2.0.0",
-        "stream-shift": "^1.0.0"
+        "end-of-stream": "1.4.1",
+        "inherits": "2.0.3",
+        "readable-stream": "2.3.6",
+        "stream-shift": "1.0.0"
       },
       "dependencies": {
         "isarray": {
@@ -3328,13 +3306,13 @@
           "integrity": "sha512-tQtKA9WIAhBF3+VLAseyMqZeBjW0AHJoxOtYqSUZNJxauErmLbVm2FW1y+J/YA9dUrAC39ITejlZWhVIwawkKw==",
           "dev": true,
           "requires": {
-            "core-util-is": "~1.0.0",
-            "inherits": "~2.0.3",
-            "isarray": "~1.0.0",
-            "process-nextick-args": "~2.0.0",
-            "safe-buffer": "~5.1.1",
-            "string_decoder": "~1.1.1",
-            "util-deprecate": "~1.0.1"
+            "core-util-is": "1.0.2",
+            "inherits": "2.0.3",
+            "isarray": "1.0.0",
+            "process-nextick-args": "2.0.0",
+            "safe-buffer": "5.1.2",
+            "string_decoder": "1.1.1",
+            "util-deprecate": "1.0.2"
           }
         },
         "string_decoder": {
@@ -3343,7 +3321,7 @@
           "integrity": "sha512-n/ShnvDi6FHbbVfviro+WojiFzv+s8MPMHBczVePfUpDJLwoLT0ht1l4YwBCbi8pJAveEEdnkHyPyTP/mzRfwg==",
           "dev": true,
           "requires": {
-            "safe-buffer": "~5.1.0"
+            "safe-buffer": "5.1.2"
           }
         }
       }
@@ -3360,8 +3338,8 @@
       "integrity": "sha1-OoOpBOVDUyh4dMVkt1SThoSamMk=",
       "dev": true,
       "requires": {
-        "jsbn": "~0.1.0",
-        "safer-buffer": "^2.1.0"
+        "jsbn": "0.1.1",
+        "safer-buffer": "2.1.2"
       }
     },
     "ee-first": {
@@ -3382,13 +3360,13 @@
       "integrity": "sha512-BsXLz5sqX8OHcsh7CqBMztyXARmGQ3LWPtGjJi6DiJHq5C/qvi9P3OqgswKSDftbu8+IoI/QDTAm2fFnQ9SZSQ==",
       "dev": true,
       "requires": {
-        "bn.js": "^4.4.0",
-        "brorand": "^1.0.1",
-        "hash.js": "^1.0.0",
-        "hmac-drbg": "^1.0.0",
-        "inherits": "^2.0.1",
-        "minimalistic-assert": "^1.0.0",
-        "minimalistic-crypto-utils": "^1.0.0"
+        "bn.js": "4.11.8",
+        "brorand": "1.1.0",
+        "hash.js": "1.1.5",
+        "hmac-drbg": "1.0.1",
+        "inherits": "2.0.3",
+        "minimalistic-assert": "1.0.1",
+        "minimalistic-crypto-utils": "1.0.1"
       }
     },
     "emojis-list": {
@@ -3409,136 +3387,7 @@
       "integrity": "sha512-1MkrZNvWTKCaigbn+W15elq2BB/L22nqrSY5DKlo3X6+vclJm8Bb5djXJBmEX6fS3+zCh/F4VBK5Z2KxJt4s2Q==",
       "dev": true,
       "requires": {
-        "once": "^1.4.0"
-      }
-    },
-    "engine.io": {
-      "version": "1.6.10",
-      "resolved": "https://registry.npmjs.org/engine.io/-/engine.io-1.6.10.tgz",
-      "integrity": "sha1-+H2E4b0h0aLsf43u8MYgVKzfsno=",
-      "dev": true,
-      "requires": {
-        "accepts": "1.1.4",
-        "base64id": "0.1.0",
-        "debug": "2.2.0",
-        "engine.io-parser": "1.2.4",
-        "ws": "1.0.1"
-      },
-      "dependencies": {
-        "accepts": {
-          "version": "1.1.4",
-          "resolved": "https://registry.npmjs.org/accepts/-/accepts-1.1.4.tgz",
-          "integrity": "sha1-1xyW99QdD+2iw4zRToonwEFY30o=",
-          "dev": true,
-          "requires": {
-            "mime-types": "~2.0.4",
-            "negotiator": "0.4.9"
-          }
-        },
-        "debug": {
-          "version": "2.2.0",
-          "resolved": "https://registry.npmjs.org/debug/-/debug-2.2.0.tgz",
-          "integrity": "sha1-+HBX6ZWxofauaklgZkE3vFbwOdo=",
-          "dev": true,
-          "requires": {
-            "ms": "0.7.1"
-          }
-        },
-        "mime-db": {
-          "version": "1.12.0",
-          "resolved": "https://registry.npmjs.org/mime-db/-/mime-db-1.12.0.tgz",
-          "integrity": "sha1-PQxjGA9FjrENMlqqN9fFiuMS6dc=",
-          "dev": true
-        },
-        "mime-types": {
-          "version": "2.0.14",
-          "resolved": "https://registry.npmjs.org/mime-types/-/mime-types-2.0.14.tgz",
-          "integrity": "sha1-MQ4VnbI+B3+Lsit0jav6SVcUCqY=",
-          "dev": true,
-          "requires": {
-            "mime-db": "~1.12.0"
-          }
-        },
-        "ms": {
-          "version": "0.7.1",
-          "resolved": "https://registry.npmjs.org/ms/-/ms-0.7.1.tgz",
-          "integrity": "sha1-nNE8A62/8ltl7/3nzoZO6VIBcJg=",
-          "dev": true
-        },
-        "negotiator": {
-          "version": "0.4.9",
-          "resolved": "https://registry.npmjs.org/negotiator/-/negotiator-0.4.9.tgz",
-          "integrity": "sha1-kuRrbbU8fkIe1koryU8IvnYw3z8=",
-          "dev": true
-        }
-      }
-    },
-    "engine.io-client": {
-      "version": "1.6.9",
-      "resolved": "https://registry.npmjs.org/engine.io-client/-/engine.io-client-1.6.9.tgz",
-      "integrity": "sha1-HWrUgEilCDyVCWlDsp0279shJAE=",
-      "dev": true,
-      "requires": {
-        "component-emitter": "1.1.2",
-        "component-inherit": "0.0.3",
-        "debug": "2.2.0",
-        "engine.io-parser": "1.2.4",
-        "has-cors": "1.1.0",
-        "indexof": "0.0.1",
-        "parsejson": "0.0.1",
-        "parseqs": "0.0.2",
-        "parseuri": "0.0.4",
-        "ws": "1.0.1",
-        "xmlhttprequest-ssl": "1.5.1",
-        "yeast": "0.1.2"
-      },
-      "dependencies": {
-        "component-emitter": {
-          "version": "1.1.2",
-          "resolved": "https://registry.npmjs.org/component-emitter/-/component-emitter-1.1.2.tgz",
-          "integrity": "sha1-KWWU8nU9qmOZbSrwjRWpURbJrsM=",
-          "dev": true
-        },
-        "debug": {
-          "version": "2.2.0",
-          "resolved": "https://registry.npmjs.org/debug/-/debug-2.2.0.tgz",
-          "integrity": "sha1-+HBX6ZWxofauaklgZkE3vFbwOdo=",
-          "dev": true,
-          "requires": {
-            "ms": "0.7.1"
-          }
-        },
-        "ms": {
-          "version": "0.7.1",
-          "resolved": "https://registry.npmjs.org/ms/-/ms-0.7.1.tgz",
-          "integrity": "sha1-nNE8A62/8ltl7/3nzoZO6VIBcJg=",
-          "dev": true
-        }
-      }
-    },
-    "engine.io-parser": {
-      "version": "1.2.4",
-      "resolved": "https://registry.npmjs.org/engine.io-parser/-/engine.io-parser-1.2.4.tgz",
-      "integrity": "sha1-4Il7C/FOeS1M0qWVBVORnFaUjEI=",
-      "dev": true,
-      "requires": {
-        "after": "0.8.1",
-        "arraybuffer.slice": "0.0.6",
-        "base64-arraybuffer": "0.1.2",
-        "blob": "0.0.4",
-        "has-binary": "0.1.6",
-        "utf8": "2.1.0"
-      },
-      "dependencies": {
-        "has-binary": {
-          "version": "0.1.6",
-          "resolved": "https://registry.npmjs.org/has-binary/-/has-binary-0.1.6.tgz",
-          "integrity": "sha1-JTJvOc+k9hath4eJTjryz7x7bhA=",
-          "dev": true,
-          "requires": {
-            "isarray": "0.0.1"
-          }
-        }
+        "once": "1.4.0"
       }
     },
     "enhanced-resolve": {
@@ -3547,9 +3396,9 @@
       "integrity": "sha512-F/7vkyTtyc/llOIn8oWclcB25KdRaiPBpZYDgJHgh/UHtpgT2p2eldQgtQnLtUvfMKPKxbRaQM/hHkvLHt1Vng==",
       "dev": true,
       "requires": {
-        "graceful-fs": "^4.1.2",
-        "memory-fs": "^0.4.0",
-        "tapable": "^1.0.0"
+        "graceful-fs": "4.1.15",
+        "memory-fs": "0.4.1",
+        "tapable": "1.1.0"
       }
     },
     "ent": {
@@ -3569,7 +3418,7 @@
       "integrity": "sha512-MfrRBDWzIWifgq6tJj60gkAwtLNb6sQPlcFrSOflcP1aFmmruKQ2wRnze/8V6kgyz7H3FF8Npzv78mZ7XLLflg==",
       "dev": true,
       "requires": {
-        "prr": "~1.0.1"
+        "prr": "1.0.1"
       }
     },
     "error-ex": {
@@ -3578,7 +3427,7 @@
       "integrity": "sha512-7dFHNmqeFSEt2ZBsCriorKnn3Z2pj+fd9kmI6QoWw4//DL+icEBfc0U7qJCisqrTsKTjw4fNFy2pW9OqStD84g==",
       "dev": true,
       "requires": {
-        "is-arrayish": "^0.2.1"
+        "is-arrayish": "0.2.1"
       }
     },
     "es-abstract": {
@@ -3587,11 +3436,11 @@
       "integrity": "sha512-C8Fx/0jFmV5IPoMOFPA9P9G5NtqW+4cOPit3MIuvR2t7Ag2K15EJTpxnHAYTzL+aYQJIESYeXZmDBfOBE1HcpA==",
       "dev": true,
       "requires": {
-        "es-to-primitive": "^1.1.1",
-        "function-bind": "^1.1.1",
-        "has": "^1.0.1",
-        "is-callable": "^1.1.3",
-        "is-regex": "^1.0.4"
+        "es-to-primitive": "1.2.0",
+        "function-bind": "1.1.1",
+        "has": "1.0.3",
+        "is-callable": "1.1.4",
+        "is-regex": "1.0.4"
       }
     },
     "es-to-primitive": {
@@ -3600,9 +3449,9 @@
       "integrity": "sha512-qZryBOJjV//LaxLTV6UC//WewneB3LcXOL9NP++ozKVXsIIIpm/2c13UDiD9Jp2eThsecw9m3jPqDwTyobcdbg==",
       "dev": true,
       "requires": {
-        "is-callable": "^1.1.4",
-        "is-date-object": "^1.0.1",
-        "is-symbol": "^1.0.2"
+        "is-callable": "1.1.4",
+        "is-date-object": "1.0.1",
+        "is-symbol": "1.0.2"
       }
     },
     "es5-ext": {
@@ -3611,9 +3460,9 @@
       "integrity": "sha512-24XxRvJXNFwEMpJb3nOkiRJKRoupmjYmOPVlI65Qy2SrtxwOTB+g6ODjBKOtwEHbYrhWRty9xxOWLNdClT2djw==",
       "dev": true,
       "requires": {
-        "es6-iterator": "~2.0.3",
-        "es6-symbol": "~3.1.1",
-        "next-tick": "1"
+        "es6-iterator": "2.0.3",
+        "es6-symbol": "3.1.1",
+        "next-tick": "1.0.0"
       }
     },
     "es6-iterator": {
@@ -3622,9 +3471,9 @@
       "integrity": "sha1-p96IkUGgWpSwhUQDstCg+/qY87c=",
       "dev": true,
       "requires": {
-        "d": "1",
-        "es5-ext": "^0.10.35",
-        "es6-symbol": "^3.1.1"
+        "d": "1.0.0",
+        "es5-ext": "0.10.46",
+        "es6-symbol": "3.1.1"
       }
     },
     "es6-map": {
@@ -3633,12 +3482,12 @@
       "integrity": "sha1-kTbgUD3MBqMBaQ8LsU/042TpSfA=",
       "dev": true,
       "requires": {
-        "d": "1",
-        "es5-ext": "~0.10.14",
-        "es6-iterator": "~2.0.1",
-        "es6-set": "~0.1.5",
-        "es6-symbol": "~3.1.1",
-        "event-emitter": "~0.3.5"
+        "d": "1.0.0",
+        "es5-ext": "0.10.46",
+        "es6-iterator": "2.0.3",
+        "es6-set": "0.1.5",
+        "es6-symbol": "3.1.1",
+        "event-emitter": "0.3.5"
       }
     },
     "es6-promise": {
@@ -3653,7 +3502,7 @@
       "integrity": "sha1-UQnWLz5W6pZ8S2NQWu8IKRyKUgM=",
       "dev": true,
       "requires": {
-        "es6-promise": "^4.0.3"
+        "es6-promise": "4.2.5"
       },
       "dependencies": {
         "es6-promise": {
@@ -3670,11 +3519,11 @@
       "integrity": "sha1-0rPsXU2ADO2BjbU40ol02wpzzLE=",
       "dev": true,
       "requires": {
-        "d": "1",
-        "es5-ext": "~0.10.14",
-        "es6-iterator": "~2.0.1",
+        "d": "1.0.0",
+        "es5-ext": "0.10.46",
+        "es6-iterator": "2.0.3",
         "es6-symbol": "3.1.1",
-        "event-emitter": "~0.3.5"
+        "event-emitter": "0.3.5"
       }
     },
     "es6-symbol": {
@@ -3683,8 +3532,8 @@
       "integrity": "sha1-vwDvT9q2uhtG7Le2KbTH7VcVzHc=",
       "dev": true,
       "requires": {
-        "d": "1",
-        "es5-ext": "~0.10.14"
+        "d": "1.0.0",
+        "es5-ext": "0.10.46"
       }
     },
     "es6-weak-map": {
@@ -3693,10 +3542,10 @@
       "integrity": "sha1-XjqzIlH/0VOKH45f+hNXdy+S2W8=",
       "dev": true,
       "requires": {
-        "d": "1",
-        "es5-ext": "^0.10.14",
-        "es6-iterator": "^2.0.1",
-        "es6-symbol": "^3.1.1"
+        "d": "1.0.0",
+        "es5-ext": "0.10.46",
+        "es6-iterator": "2.0.3",
+        "es6-symbol": "3.1.1"
       }
     },
     "escape-html": {
@@ -3717,11 +3566,11 @@
       "integrity": "sha1-WltTr0aTEQvrsIZ6o0MN07cKEBg=",
       "dev": true,
       "requires": {
-        "esprima": "^2.7.1",
-        "estraverse": "^1.9.1",
-        "esutils": "^2.0.2",
-        "optionator": "^0.8.1",
-        "source-map": "~0.2.0"
+        "esprima": "2.7.3",
+        "estraverse": "1.9.3",
+        "esutils": "2.0.2",
+        "optionator": "0.8.2",
+        "source-map": "0.2.0"
       },
       "dependencies": {
         "estraverse": {
@@ -3737,7 +3586,7 @@
           "dev": true,
           "optional": true,
           "requires": {
-            "amdefine": ">=0.0.4"
+            "amdefine": "1.0.1"
           }
         }
       }
@@ -3748,10 +3597,10 @@
       "integrity": "sha1-4Bl16BJ4GhY6ba392AOY3GTIicM=",
       "dev": true,
       "requires": {
-        "es6-map": "^0.1.3",
-        "es6-weak-map": "^2.0.1",
-        "esrecurse": "^4.1.0",
-        "estraverse": "^4.1.1"
+        "es6-map": "0.1.5",
+        "es6-weak-map": "2.0.2",
+        "esrecurse": "4.2.1",
+        "estraverse": "4.2.0"
       }
     },
     "eslint": {
@@ -3760,41 +3609,41 @@
       "integrity": "sha1-yPxiAcf0DdCJQbh8CFdnOGpnmsw=",
       "dev": true,
       "requires": {
-        "babel-code-frame": "^6.16.0",
-        "chalk": "^1.1.3",
-        "concat-stream": "^1.5.2",
-        "debug": "^2.1.1",
-        "doctrine": "^2.0.0",
-        "escope": "^3.6.0",
-        "espree": "^3.4.0",
-        "esquery": "^1.0.0",
-        "estraverse": "^4.2.0",
-        "esutils": "^2.0.2",
-        "file-entry-cache": "^2.0.0",
-        "glob": "^7.0.3",
-        "globals": "^9.14.0",
-        "ignore": "^3.2.0",
-        "imurmurhash": "^0.1.4",
-        "inquirer": "^0.12.0",
-        "is-my-json-valid": "^2.10.0",
-        "is-resolvable": "^1.0.0",
-        "js-yaml": "^3.5.1",
-        "json-stable-stringify": "^1.0.0",
-        "levn": "^0.3.0",
-        "lodash": "^4.0.0",
-        "mkdirp": "^0.5.0",
-        "natural-compare": "^1.4.0",
-        "optionator": "^0.8.2",
-        "path-is-inside": "^1.0.1",
-        "pluralize": "^1.2.1",
-        "progress": "^1.1.8",
-        "require-uncached": "^1.0.2",
-        "shelljs": "^0.7.5",
-        "strip-bom": "^3.0.0",
-        "strip-json-comments": "~2.0.1",
-        "table": "^3.7.8",
-        "text-table": "~0.2.0",
-        "user-home": "^2.0.0"
+        "babel-code-frame": "6.26.0",
+        "chalk": "1.1.3",
+        "concat-stream": "1.6.2",
+        "debug": "2.6.9",
+        "doctrine": "2.1.0",
+        "escope": "3.6.0",
+        "espree": "3.5.4",
+        "esquery": "1.0.1",
+        "estraverse": "4.2.0",
+        "esutils": "2.0.2",
+        "file-entry-cache": "2.0.0",
+        "glob": "7.1.3",
+        "globals": "9.18.0",
+        "ignore": "3.3.10",
+        "imurmurhash": "0.1.4",
+        "inquirer": "0.12.0",
+        "is-my-json-valid": "2.19.0",
+        "is-resolvable": "1.1.0",
+        "js-yaml": "3.7.0",
+        "json-stable-stringify": "1.0.1",
+        "levn": "0.3.0",
+        "lodash": "4.17.11",
+        "mkdirp": "0.5.1",
+        "natural-compare": "1.4.0",
+        "optionator": "0.8.2",
+        "path-is-inside": "1.0.2",
+        "pluralize": "1.2.1",
+        "progress": "1.1.8",
+        "require-uncached": "1.0.3",
+        "shelljs": "0.7.8",
+        "strip-bom": "3.0.0",
+        "strip-json-comments": "2.0.1",
+        "table": "3.8.3",
+        "text-table": "0.2.0",
+        "user-home": "2.0.0"
       },
       "dependencies": {
         "glob": {
@@ -3803,12 +3652,12 @@
           "integrity": "sha512-vcfuiIxogLV4DlGBHIUOwI0IbrJ8HWPc4MU7HzviGeNho/UJDfi6B5p3sHeWIQ0KGIU0Jpxi5ZHxemQfLkkAwQ==",
           "dev": true,
           "requires": {
-            "fs.realpath": "^1.0.0",
-            "inflight": "^1.0.4",
-            "inherits": "2",
-            "minimatch": "^3.0.4",
-            "once": "^1.3.0",
-            "path-is-absolute": "^1.0.0"
+            "fs.realpath": "1.0.0",
+            "inflight": "1.0.6",
+            "inherits": "2.0.3",
+            "minimatch": "3.0.4",
+            "once": "1.4.0",
+            "path-is-absolute": "1.0.1"
           }
         },
         "minimatch": {
@@ -3817,7 +3666,7 @@
           "integrity": "sha512-yJHVQEhyqPLUTgt9B83PXu6W3rx4MvvHvSUvToogpwoGDOUQ+yDrR0HRot+yOCdCO7u4hX3pWft6kWBBcqh0UA==",
           "dev": true,
           "requires": {
-            "brace-expansion": "^1.1.7"
+            "brace-expansion": "1.1.11"
           }
         },
         "user-home": {
@@ -3826,7 +3675,7 @@
           "integrity": "sha1-nHC/2Babwdy/SGBODwS4tJzenp8=",
           "dev": true,
           "requires": {
-            "os-homedir": "^1.0.0"
+            "os-homedir": "1.0.2"
           }
         }
       }
@@ -3849,8 +3698,8 @@
       "integrity": "sha512-sfmTqJfPSizWu4aymbPr4Iidp5yKm8yDkHp+Ir3YiTHiiDfxh69mOUsmiqW6RZ9zRXFaF64GtYmN7e+8GHBv6Q==",
       "dev": true,
       "requires": {
-        "debug": "^2.6.9",
-        "resolve": "^1.5.0"
+        "debug": "2.6.9",
+        "resolve": "1.8.1"
       }
     },
     "eslint-module-utils": {
@@ -3859,8 +3708,8 @@
       "integrity": "sha1-snA2LNiLGkitMIl2zn+lTphBF0Y=",
       "dev": true,
       "requires": {
-        "debug": "^2.6.8",
-        "pkg-dir": "^1.0.0"
+        "debug": "2.6.9",
+        "pkg-dir": "1.0.0"
       },
       "dependencies": {
         "find-up": {
@@ -3869,8 +3718,8 @@
           "integrity": "sha1-ay6YIrGizgpgq2TWEOzK1TyyTQ8=",
           "dev": true,
           "requires": {
-            "path-exists": "^2.0.0",
-            "pinkie-promise": "^2.0.0"
+            "path-exists": "2.1.0",
+            "pinkie-promise": "2.0.1"
           }
         },
         "path-exists": {
@@ -3879,7 +3728,7 @@
           "integrity": "sha1-D+tsZPD8UY2adU3V77YscCJ2H0s=",
           "dev": true,
           "requires": {
-            "pinkie-promise": "^2.0.0"
+            "pinkie-promise": "2.0.1"
           }
         },
         "pkg-dir": {
@@ -3888,7 +3737,7 @@
           "integrity": "sha1-ektQio1bstYp1EcFb/TpyTFM89Q=",
           "dev": true,
           "requires": {
-            "find-up": "^1.0.0"
+            "find-up": "1.1.2"
           }
         }
       }
@@ -3899,16 +3748,16 @@
       "integrity": "sha512-FpuRtniD/AY6sXByma2Wr0TXvXJ4nA/2/04VPlfpmUDPOpOY264x+ILiwnrk/k4RINgDAyFZByxqPUbSQ5YE7g==",
       "dev": true,
       "requires": {
-        "contains-path": "^0.1.0",
-        "debug": "^2.6.8",
+        "contains-path": "0.1.0",
+        "debug": "2.6.9",
         "doctrine": "1.5.0",
-        "eslint-import-resolver-node": "^0.3.1",
-        "eslint-module-utils": "^2.2.0",
-        "has": "^1.0.1",
-        "lodash": "^4.17.4",
-        "minimatch": "^3.0.3",
-        "read-pkg-up": "^2.0.0",
-        "resolve": "^1.6.0"
+        "eslint-import-resolver-node": "0.3.2",
+        "eslint-module-utils": "2.2.0",
+        "has": "1.0.3",
+        "lodash": "4.17.11",
+        "minimatch": "3.0.4",
+        "read-pkg-up": "2.0.0",
+        "resolve": "1.8.1"
       },
       "dependencies": {
         "doctrine": {
@@ -3917,8 +3766,8 @@
           "integrity": "sha1-N53Ocw9hZvds76TmcHoVmwLFpvo=",
           "dev": true,
           "requires": {
-            "esutils": "^2.0.2",
-            "isarray": "^1.0.0"
+            "esutils": "2.0.2",
+            "isarray": "1.0.0"
           }
         },
         "isarray": {
@@ -3933,7 +3782,7 @@
           "integrity": "sha512-yJHVQEhyqPLUTgt9B83PXu6W3rx4MvvHvSUvToogpwoGDOUQ+yDrR0HRot+yOCdCO7u4hX3pWft6kWBBcqh0UA==",
           "dev": true,
           "requires": {
-            "brace-expansion": "^1.1.7"
+            "brace-expansion": "1.1.11"
           }
         }
       }
@@ -3944,10 +3793,10 @@
       "integrity": "sha512-vIUQPuwbVYdz/CYnlTLsJrRy7iXHQjdEe5wz0XhhdTym3IInM/zZLlPf9nZ2mThsH0QcsieCOWs2vOeCy/22LQ==",
       "dev": true,
       "requires": {
-        "ignore": "^3.0.11",
-        "minimatch": "^3.0.2",
-        "object-assign": "^4.0.1",
-        "resolve": "^1.1.7",
+        "ignore": "3.3.10",
+        "minimatch": "3.0.4",
+        "object-assign": "4.1.1",
+        "resolve": "1.8.1",
         "semver": "5.3.0"
       },
       "dependencies": {
@@ -3957,7 +3806,7 @@
           "integrity": "sha512-yJHVQEhyqPLUTgt9B83PXu6W3rx4MvvHvSUvToogpwoGDOUQ+yDrR0HRot+yOCdCO7u4hX3pWft6kWBBcqh0UA==",
           "dev": true,
           "requires": {
-            "brace-expansion": "^1.1.7"
+            "brace-expansion": "1.1.11"
           }
         },
         "semver": {
@@ -3986,8 +3835,8 @@
       "integrity": "sha512-1G6UTDi7Jc1ELFwnR58HV4fK9OQK4S6N985f166xqXxpjU6plxFISJa2Ba9KCQuFa8RCnj/lSFJbHo7UFDBnUA==",
       "dev": true,
       "requires": {
-        "esrecurse": "^4.1.0",
-        "estraverse": "^4.1.1"
+        "esrecurse": "4.2.1",
+        "estraverse": "4.2.0"
       }
     },
     "eslint-watch": {
@@ -3996,17 +3845,17 @@
       "integrity": "sha512-6iEMRwo6RUpSaYyU7547qWQbgUKSYtkn4eGId/hZJvi+gMnRVeNfIzv/HAOPUmH6y53p1Ks9oNvWm/xZh4RPGQ==",
       "dev": true,
       "requires": {
-        "babel-polyfill": "^6.20.0",
-        "bluebird": "^3.5.1",
-        "chalk": "^2.1.0",
-        "chokidar": "^2.0.0",
-        "debug": "^3.0.1",
-        "keypress": "^0.2.1",
-        "lodash": "^4.17.4",
-        "optionator": "^0.8.2",
-        "source-map-support": "^0.5.3",
-        "strip-ansi": "^4.0.0",
-        "text-table": "^0.2.0",
+        "babel-polyfill": "6.26.0",
+        "bluebird": "3.5.2",
+        "chalk": "2.4.1",
+        "chokidar": "2.0.4",
+        "debug": "3.2.6",
+        "keypress": "0.2.1",
+        "lodash": "4.17.11",
+        "optionator": "0.8.2",
+        "source-map-support": "0.5.9",
+        "strip-ansi": "4.0.0",
+        "text-table": "0.2.0",
         "unicons": "0.0.3"
       },
       "dependencies": {
@@ -4022,7 +3871,7 @@
           "integrity": "sha512-VT0ZI6kZRdTh8YyJw3SMbYm/u+NqfsAxEpWO0Pf9sq8/e94WxxOpPKx9FR1FlyCtOVDNOQ+8ntlqFxiRc+r5qA==",
           "dev": true,
           "requires": {
-            "color-convert": "^1.9.0"
+            "color-convert": "1.9.3"
           }
         },
         "anymatch": {
@@ -4031,8 +3880,8 @@
           "integrity": "sha512-5teOsQWABXHHBFP9y3skS5P3d/WfWXpv3FUpy+LorMrNYaT9pI4oLMQX7jzQ2KklNpGpWHzdCXTDT2Y3XGlZBw==",
           "dev": true,
           "requires": {
-            "micromatch": "^3.1.4",
-            "normalize-path": "^2.1.1"
+            "micromatch": "3.1.10",
+            "normalize-path": "2.1.1"
           }
         },
         "arr-diff": {
@@ -4053,16 +3902,16 @@
           "integrity": "sha512-aNdbnj9P8PjdXU4ybaWLK2IF3jc/EoDYbC7AazW6to3TRsfXxscC9UXOB5iDiEQrkyIbWp2SLQda4+QAa7nc3w==",
           "dev": true,
           "requires": {
-            "arr-flatten": "^1.1.0",
-            "array-unique": "^0.3.2",
-            "extend-shallow": "^2.0.1",
-            "fill-range": "^4.0.0",
-            "isobject": "^3.0.1",
-            "repeat-element": "^1.1.2",
-            "snapdragon": "^0.8.1",
-            "snapdragon-node": "^2.0.1",
-            "split-string": "^3.0.2",
-            "to-regex": "^3.0.1"
+            "arr-flatten": "1.1.0",
+            "array-unique": "0.3.2",
+            "extend-shallow": "2.0.1",
+            "fill-range": "4.0.0",
+            "isobject": "3.0.1",
+            "repeat-element": "1.1.3",
+            "snapdragon": "0.8.2",
+            "snapdragon-node": "2.1.1",
+            "split-string": "3.1.0",
+            "to-regex": "3.0.2"
           },
           "dependencies": {
             "extend-shallow": {
@@ -4071,7 +3920,7 @@
               "integrity": "sha1-Ua99YUrZqfYQ6huvu5idaxxWiQ8=",
               "dev": true,
               "requires": {
-                "is-extendable": "^0.1.0"
+                "is-extendable": "0.1.1"
               }
             }
           }
@@ -4082,9 +3931,9 @@
           "integrity": "sha512-ObN6h1v2fTJSmUXoS3nMQ92LbDK9be4TV+6G+omQlGJFdcUX5heKi1LZ1YnRMIgwTLEj3E24bT6tYni50rlCfQ==",
           "dev": true,
           "requires": {
-            "ansi-styles": "^3.2.1",
-            "escape-string-regexp": "^1.0.5",
-            "supports-color": "^5.3.0"
+            "ansi-styles": "3.2.1",
+            "escape-string-regexp": "1.0.5",
+            "supports-color": "5.5.0"
           }
         },
         "chokidar": {
@@ -4093,19 +3942,18 @@
           "integrity": "sha512-z9n7yt9rOvIJrMhvDtDictKrkFHeihkNl6uWMmZlmL6tJtX9Cs+87oK+teBx+JIgzvbX3yZHT3eF8vpbDxHJXQ==",
           "dev": true,
           "requires": {
-            "anymatch": "^2.0.0",
-            "async-each": "^1.0.0",
-            "braces": "^2.3.0",
-            "fsevents": "^1.2.2",
-            "glob-parent": "^3.1.0",
-            "inherits": "^2.0.1",
-            "is-binary-path": "^1.0.0",
-            "is-glob": "^4.0.0",
-            "lodash.debounce": "^4.0.8",
-            "normalize-path": "^2.1.1",
-            "path-is-absolute": "^1.0.0",
-            "readdirp": "^2.0.0",
-            "upath": "^1.0.5"
+            "anymatch": "2.0.0",
+            "async-each": "1.0.1",
+            "braces": "2.3.2",
+            "glob-parent": "3.1.0",
+            "inherits": "2.0.3",
+            "is-binary-path": "1.0.1",
+            "is-glob": "4.0.0",
+            "lodash.debounce": "4.0.8",
+            "normalize-path": "2.1.1",
+            "path-is-absolute": "1.0.1",
+            "readdirp": "2.2.1",
+            "upath": "1.1.0"
           }
         },
         "debug": {
@@ -4114,7 +3962,7 @@
           "integrity": "sha512-mel+jf7nrtEl5Pn1Qx46zARXKDpBbvzezse7p7LqINmdoIk8PYP5SySaxEmYv6TZ0JyEKA1hsCId6DIhgITtWQ==",
           "dev": true,
           "requires": {
-            "ms": "^2.1.1"
+            "ms": "2.1.1"
           },
           "dependencies": {
             "ms": {
@@ -4131,13 +3979,13 @@
           "integrity": "sha1-t3c14xXOMPa27/D4OwQVGiJEliI=",
           "dev": true,
           "requires": {
-            "debug": "^2.3.3",
-            "define-property": "^0.2.5",
-            "extend-shallow": "^2.0.1",
-            "posix-character-classes": "^0.1.0",
-            "regex-not": "^1.0.0",
-            "snapdragon": "^0.8.1",
-            "to-regex": "^3.0.1"
+            "debug": "2.6.9",
+            "define-property": "0.2.5",
+            "extend-shallow": "2.0.1",
+            "posix-character-classes": "0.1.1",
+            "regex-not": "1.0.2",
+            "snapdragon": "0.8.2",
+            "to-regex": "3.0.2"
           },
           "dependencies": {
             "debug": {
@@ -4155,7 +4003,7 @@
               "integrity": "sha1-w1se+RjsPJkPmlvFe+BKrOxcgRY=",
               "dev": true,
               "requires": {
-                "is-descriptor": "^0.1.0"
+                "is-descriptor": "0.1.6"
               }
             },
             "extend-shallow": {
@@ -4164,7 +4012,7 @@
               "integrity": "sha1-Ua99YUrZqfYQ6huvu5idaxxWiQ8=",
               "dev": true,
               "requires": {
-                "is-extendable": "^0.1.0"
+                "is-extendable": "0.1.1"
               }
             },
             "is-accessor-descriptor": {
@@ -4173,7 +4021,7 @@
               "integrity": "sha1-qeEss66Nh2cn7u84Q/igiXtcmNY=",
               "dev": true,
               "requires": {
-                "kind-of": "^3.0.2"
+                "kind-of": "3.2.2"
               },
               "dependencies": {
                 "kind-of": {
@@ -4182,7 +4030,7 @@
                   "integrity": "sha1-MeohpzS6ubuw8yRm2JOupR5KPGQ=",
                   "dev": true,
                   "requires": {
-                    "is-buffer": "^1.1.5"
+                    "is-buffer": "1.1.6"
                   }
                 }
               }
@@ -4193,7 +4041,7 @@
               "integrity": "sha1-C17mSDiOLIYCgueT8YVv7D8wG1Y=",
               "dev": true,
               "requires": {
-                "kind-of": "^3.0.2"
+                "kind-of": "3.2.2"
               },
               "dependencies": {
                 "kind-of": {
@@ -4202,7 +4050,7 @@
                   "integrity": "sha1-MeohpzS6ubuw8yRm2JOupR5KPGQ=",
                   "dev": true,
                   "requires": {
-                    "is-buffer": "^1.1.5"
+                    "is-buffer": "1.1.6"
                   }
                 }
               }
@@ -4213,9 +4061,9 @@
               "integrity": "sha512-avDYr0SB3DwO9zsMov0gKCESFYqCnE4hq/4z3TdUlukEy5t9C0YRq7HLrsN52NAcqXKaepeCD0n+B0arnVG3Hg==",
               "dev": true,
               "requires": {
-                "is-accessor-descriptor": "^0.1.6",
-                "is-data-descriptor": "^0.1.4",
-                "kind-of": "^5.0.0"
+                "is-accessor-descriptor": "0.1.6",
+                "is-data-descriptor": "0.1.4",
+                "kind-of": "5.1.0"
               }
             },
             "kind-of": {
@@ -4232,14 +4080,14 @@
           "integrity": "sha512-Nmb6QXkELsuBr24CJSkilo6UHHgbekK5UiZgfE6UHD3Eb27YC6oD+bhcT+tJ6cl8dmsgdQxnWlcry8ksBIBLpw==",
           "dev": true,
           "requires": {
-            "array-unique": "^0.3.2",
-            "define-property": "^1.0.0",
-            "expand-brackets": "^2.1.4",
-            "extend-shallow": "^2.0.1",
-            "fragment-cache": "^0.2.1",
-            "regex-not": "^1.0.0",
-            "snapdragon": "^0.8.1",
-            "to-regex": "^3.0.1"
+            "array-unique": "0.3.2",
+            "define-property": "1.0.0",
+            "expand-brackets": "2.1.4",
+            "extend-shallow": "2.0.1",
+            "fragment-cache": "0.2.1",
+            "regex-not": "1.0.2",
+            "snapdragon": "0.8.2",
+            "to-regex": "3.0.2"
           },
           "dependencies": {
             "define-property": {
@@ -4248,7 +4096,7 @@
               "integrity": "sha1-dp66rz9KY6rTr56NMEybvnm/sOY=",
               "dev": true,
               "requires": {
-                "is-descriptor": "^1.0.0"
+                "is-descriptor": "1.0.2"
               }
             },
             "extend-shallow": {
@@ -4257,7 +4105,7 @@
               "integrity": "sha1-Ua99YUrZqfYQ6huvu5idaxxWiQ8=",
               "dev": true,
               "requires": {
-                "is-extendable": "^0.1.0"
+                "is-extendable": "0.1.1"
               }
             }
           }
@@ -4268,10 +4116,10 @@
           "integrity": "sha1-1USBHUKPmOsGpj3EAtJAPDKMOPc=",
           "dev": true,
           "requires": {
-            "extend-shallow": "^2.0.1",
-            "is-number": "^3.0.0",
-            "repeat-string": "^1.6.1",
-            "to-regex-range": "^2.1.0"
+            "extend-shallow": "2.0.1",
+            "is-number": "3.0.0",
+            "repeat-string": "1.6.1",
+            "to-regex-range": "2.1.1"
           },
           "dependencies": {
             "extend-shallow": {
@@ -4280,7 +4128,7 @@
               "integrity": "sha1-Ua99YUrZqfYQ6huvu5idaxxWiQ8=",
               "dev": true,
               "requires": {
-                "is-extendable": "^0.1.0"
+                "is-extendable": "0.1.1"
               }
             }
           }
@@ -4291,8 +4139,8 @@
           "integrity": "sha1-nmr2KZ2NO9K9QEMIMr0RPfkGxa4=",
           "dev": true,
           "requires": {
-            "is-glob": "^3.1.0",
-            "path-dirname": "^1.0.0"
+            "is-glob": "3.1.0",
+            "path-dirname": "1.0.2"
           },
           "dependencies": {
             "is-glob": {
@@ -4301,7 +4149,7 @@
               "integrity": "sha1-e6WuJCF4BKxwcHuWkiVnSGzD6Eo=",
               "dev": true,
               "requires": {
-                "is-extglob": "^2.1.0"
+                "is-extglob": "2.1.1"
               }
             }
           }
@@ -4318,7 +4166,7 @@
           "integrity": "sha512-m5hnHTkcVsPfqx3AKlyttIPb7J+XykHvJP2B9bZDjlhLIoEq4XoK64Vg7boZlVWYK6LUY94dYPEE7Lh0ZkZKcQ==",
           "dev": true,
           "requires": {
-            "kind-of": "^6.0.0"
+            "kind-of": "6.0.2"
           }
         },
         "is-data-descriptor": {
@@ -4327,7 +4175,7 @@
           "integrity": "sha512-jbRXy1FmtAoCjQkVmIVYwuuqDFUbaOeDjmed1tOGPrsMhtJA4rD9tkgA0F1qJ3gRFRXcHYVkdeaP50Q5rE/jLQ==",
           "dev": true,
           "requires": {
-            "kind-of": "^6.0.0"
+            "kind-of": "6.0.2"
           }
         },
         "is-descriptor": {
@@ -4336,9 +4184,9 @@
           "integrity": "sha512-2eis5WqQGV7peooDyLmNEPUrps9+SXX5c9pL3xEB+4e9HnGuDa7mB7kHxHw4CbqS9k1T2hOH3miL8n8WtiYVtg==",
           "dev": true,
           "requires": {
-            "is-accessor-descriptor": "^1.0.0",
-            "is-data-descriptor": "^1.0.0",
-            "kind-of": "^6.0.2"
+            "is-accessor-descriptor": "1.0.0",
+            "is-data-descriptor": "1.0.0",
+            "kind-of": "6.0.2"
           }
         },
         "is-extglob": {
@@ -4353,7 +4201,7 @@
           "integrity": "sha1-lSHHaEXMJhCoUgPd8ICpWML/q8A=",
           "dev": true,
           "requires": {
-            "is-extglob": "^2.1.1"
+            "is-extglob": "2.1.1"
           }
         },
         "is-number": {
@@ -4362,7 +4210,7 @@
           "integrity": "sha1-JP1iAaR4LPUFYcgQJ2r8fRLXEZU=",
           "dev": true,
           "requires": {
-            "kind-of": "^3.0.2"
+            "kind-of": "3.2.2"
           },
           "dependencies": {
             "kind-of": {
@@ -4371,7 +4219,7 @@
               "integrity": "sha1-MeohpzS6ubuw8yRm2JOupR5KPGQ=",
               "dev": true,
               "requires": {
-                "is-buffer": "^1.1.5"
+                "is-buffer": "1.1.6"
               }
             }
           }
@@ -4394,19 +4242,19 @@
           "integrity": "sha512-MWikgl9n9M3w+bpsY3He8L+w9eF9338xRl8IAO5viDizwSzziFEyUzo2xrrloB64ADbTf8uA8vRqqttDTOmccg==",
           "dev": true,
           "requires": {
-            "arr-diff": "^4.0.0",
-            "array-unique": "^0.3.2",
-            "braces": "^2.3.1",
-            "define-property": "^2.0.2",
-            "extend-shallow": "^3.0.2",
-            "extglob": "^2.0.4",
-            "fragment-cache": "^0.2.1",
-            "kind-of": "^6.0.2",
-            "nanomatch": "^1.2.9",
-            "object.pick": "^1.3.0",
-            "regex-not": "^1.0.0",
-            "snapdragon": "^0.8.1",
-            "to-regex": "^3.0.2"
+            "arr-diff": "4.0.0",
+            "array-unique": "0.3.2",
+            "braces": "2.3.2",
+            "define-property": "2.0.2",
+            "extend-shallow": "3.0.2",
+            "extglob": "2.0.4",
+            "fragment-cache": "0.2.1",
+            "kind-of": "6.0.2",
+            "nanomatch": "1.2.13",
+            "object.pick": "1.3.0",
+            "regex-not": "1.0.2",
+            "snapdragon": "0.8.2",
+            "to-regex": "3.0.2"
           }
         },
         "source-map": {
@@ -4421,8 +4269,8 @@
           "integrity": "sha512-gR6Rw4MvUlYy83vP0vxoVNzM6t8MUXqNuRsuBmBHQDu1Fh6X015FrLdgoDKcNdkwGubozq0P4N0Q37UyFVr1EA==",
           "dev": true,
           "requires": {
-            "buffer-from": "^1.0.0",
-            "source-map": "^0.6.0"
+            "buffer-from": "1.1.1",
+            "source-map": "0.6.1"
           }
         },
         "strip-ansi": {
@@ -4431,7 +4279,7 @@
           "integrity": "sha1-qEeQIusaw2iocTibY1JixQXuNo8=",
           "dev": true,
           "requires": {
-            "ansi-regex": "^3.0.0"
+            "ansi-regex": "3.0.0"
           }
         },
         "supports-color": {
@@ -4440,7 +4288,7 @@
           "integrity": "sha512-QjVjwdXIt408MIiAqCX4oUKsgU2EqAGzs2Ppkm4aQYbjm+ZEWEcW4SfFNTr4uMNZma0ey4f5lgLrkB0aX0QMow==",
           "dev": true,
           "requires": {
-            "has-flag": "^3.0.0"
+            "has-flag": "3.0.0"
           }
         }
       }
@@ -4451,8 +4299,8 @@
       "integrity": "sha512-yAcIQxtmMiB/jL32dzEp2enBeidsB7xWPLNiw3IIkpVds1P+h7qF9YwJq1yUNzp2OKXgAprs4F61ih66UsoD1A==",
       "dev": true,
       "requires": {
-        "acorn": "^5.5.0",
-        "acorn-jsx": "^3.0.0"
+        "acorn": "5.7.3",
+        "acorn-jsx": "3.0.1"
       }
     },
     "esprima": {
@@ -4467,7 +4315,7 @@
       "integrity": "sha512-SmiyZ5zIWH9VM+SRUReLS5Q8a7GxtRdxEBVZpm98rJM7Sb+A9DVCndXfkeFUd3byderg+EbDkfnevfCwynWaNA==",
       "dev": true,
       "requires": {
-        "estraverse": "^4.0.0"
+        "estraverse": "4.2.0"
       }
     },
     "esrecurse": {
@@ -4476,7 +4324,7 @@
       "integrity": "sha512-64RBB++fIOAXPw3P9cy89qfMlvZEXZkqqJkjqqXIvzP5ezRZjW+lPWjw35UX/3EhUPFYbg5ER4JYgDw4007/DQ==",
       "dev": true,
       "requires": {
-        "estraverse": "^4.1.0"
+        "estraverse": "4.2.0"
       }
     },
     "estraverse": {
@@ -4503,8 +4351,8 @@
       "integrity": "sha1-34xp7vFkeSPHFXuc6DhAYQsCzDk=",
       "dev": true,
       "requires": {
-        "d": "1",
-        "es5-ext": "~0.10.14"
+        "d": "1.0.0",
+        "es5-ext": "0.10.46"
       }
     },
     "eventemitter2": {
@@ -4531,7 +4379,7 @@
       "integrity": "sha512-4Ln17+vVT0k8aWq+t/bF5arcS3EpT9gYtW66EPacdj/mAFevznsnyoHLPy2BA8gbIQeIHoPsvwmfBftfcG//BQ==",
       "dev": true,
       "requires": {
-        "original": "^1.0.0"
+        "original": "1.0.2"
       }
     },
     "evp_bytestokey": {
@@ -4540,8 +4388,8 @@
       "integrity": "sha512-/f2Go4TognH/KvCISP7OUsHn85hT9nUkxxA9BEWxFn+Oj9o8ZNLm/40hdlgSLyuOimsrTKLUMEorQexp/aPQeA==",
       "dev": true,
       "requires": {
-        "md5.js": "^1.3.4",
-        "safe-buffer": "^5.1.1"
+        "md5.js": "1.3.5",
+        "safe-buffer": "5.1.2"
       }
     },
     "execa": {
@@ -4550,13 +4398,13 @@
       "integrity": "sha512-7XOMnz8Ynx1gGo/3hyV9loYNPWM94jG3+3T3Y8tsfSstFmETmENCMU/A/zj8Lyaj1lkgEepKepvd6240tBRvlw==",
       "dev": true,
       "requires": {
-        "cross-spawn": "^6.0.0",
-        "get-stream": "^3.0.0",
-        "is-stream": "^1.1.0",
-        "npm-run-path": "^2.0.0",
-        "p-finally": "^1.0.0",
-        "signal-exit": "^3.0.0",
-        "strip-eof": "^1.0.0"
+        "cross-spawn": "6.0.5",
+        "get-stream": "3.0.0",
+        "is-stream": "1.1.0",
+        "npm-run-path": "2.0.2",
+        "p-finally": "1.0.0",
+        "signal-exit": "3.0.2",
+        "strip-eof": "1.0.0"
       },
       "dependencies": {
         "cross-spawn": {
@@ -4565,11 +4413,11 @@
           "integrity": "sha512-eTVLrBSt7fjbDygz805pMnstIs2VTBNkRm0qxZd+M7A5XDdxVRWO5MxGBXZhjY4cqLYLdtrGqRf8mBPmzwSpWQ==",
           "dev": true,
           "requires": {
-            "nice-try": "^1.0.4",
-            "path-key": "^2.0.1",
-            "semver": "^5.5.0",
-            "shebang-command": "^1.2.0",
-            "which": "^1.2.9"
+            "nice-try": "1.0.5",
+            "path-key": "2.0.1",
+            "semver": "5.6.0",
+            "shebang-command": "1.2.0",
+            "which": "1.3.1"
           }
         },
         "which": {
@@ -4578,7 +4426,7 @@
           "integrity": "sha512-HxJdYWq1MTIQbJ3nw0cqssHoTNU267KlrDuGZ1WYlxDStUtKUhOaJmh112/TZmHxxUfuJqPXSOm7tDyas0OSIQ==",
           "dev": true,
           "requires": {
-            "isexe": "^2.0.0"
+            "isexe": "2.0.0"
           }
         }
       }
@@ -4600,9 +4448,9 @@
       "integrity": "sha1-SIsdHSRRyz06axks/AMPRMWFX+o=",
       "dev": true,
       "requires": {
-        "array-slice": "^0.2.3",
-        "array-unique": "^0.2.1",
-        "braces": "^0.1.2"
+        "array-slice": "0.2.3",
+        "array-unique": "0.2.1",
+        "braces": "0.1.5"
       },
       "dependencies": {
         "braces": {
@@ -4611,7 +4459,7 @@
           "integrity": "sha1-wIVxEIUpHYt1/ddOqw+FlygHEeY=",
           "dev": true,
           "requires": {
-            "expand-range": "^0.1.0"
+            "expand-range": "0.1.1"
           }
         },
         "expand-range": {
@@ -4620,8 +4468,8 @@
           "integrity": "sha1-TLjtoJk8pW+k9B/ELzy7TMrf8EQ=",
           "dev": true,
           "requires": {
-            "is-number": "^0.1.1",
-            "repeat-string": "^0.2.2"
+            "is-number": "0.1.1",
+            "repeat-string": "0.2.2"
           }
         },
         "is-number": {
@@ -4643,8 +4491,9 @@
       "resolved": "https://registry.npmjs.org/expand-brackets/-/expand-brackets-0.1.5.tgz",
       "integrity": "sha1-3wcoTjQqgHzXM6xa9yQR5YHRF3s=",
       "dev": true,
+      "optional": true,
       "requires": {
-        "is-posix-bracket": "^0.1.0"
+        "is-posix-bracket": "0.1.1"
       }
     },
     "expand-range": {
@@ -4652,8 +4501,9 @@
       "resolved": "https://registry.npmjs.org/expand-range/-/expand-range-1.8.2.tgz",
       "integrity": "sha1-opnv/TNf4nIeuujiV+x5ZE/IUzc=",
       "dev": true,
+      "optional": true,
       "requires": {
-        "fill-range": "^2.1.0"
+        "fill-range": "2.2.4"
       }
     },
     "express": {
@@ -4662,36 +4512,36 @@
       "integrity": "sha512-j12Uuyb4FMrd/qQAm6uCHAkPtO8FDTRJZBDd5D2KOL2eLaz1yUNdUB/NOIyq0iU4q4cFarsUCrnFDPBcnksuOg==",
       "dev": true,
       "requires": {
-        "accepts": "~1.3.5",
+        "accepts": "1.3.5",
         "array-flatten": "1.1.1",
         "body-parser": "1.18.3",
         "content-disposition": "0.5.2",
-        "content-type": "~1.0.4",
+        "content-type": "1.0.4",
         "cookie": "0.3.1",
         "cookie-signature": "1.0.6",
         "debug": "2.6.9",
-        "depd": "~1.1.2",
-        "encodeurl": "~1.0.2",
-        "escape-html": "~1.0.3",
-        "etag": "~1.8.1",
+        "depd": "1.1.2",
+        "encodeurl": "1.0.2",
+        "escape-html": "1.0.3",
+        "etag": "1.8.1",
         "finalhandler": "1.1.1",
         "fresh": "0.5.2",
         "merge-descriptors": "1.0.1",
-        "methods": "~1.1.2",
-        "on-finished": "~2.3.0",
-        "parseurl": "~1.3.2",
+        "methods": "1.1.2",
+        "on-finished": "2.3.0",
+        "parseurl": "1.3.2",
         "path-to-regexp": "0.1.7",
-        "proxy-addr": "~2.0.4",
+        "proxy-addr": "2.0.4",
         "qs": "6.5.2",
-        "range-parser": "~1.2.0",
+        "range-parser": "1.2.0",
         "safe-buffer": "5.1.2",
         "send": "0.16.2",
         "serve-static": "1.13.2",
         "setprototypeof": "1.1.0",
-        "statuses": "~1.4.0",
-        "type-is": "~1.6.16",
+        "statuses": "1.4.0",
+        "type-is": "1.6.16",
         "utils-merge": "1.0.1",
-        "vary": "~1.1.2"
+        "vary": "1.1.2"
       }
     },
     "extend": {
@@ -4706,8 +4556,8 @@
       "integrity": "sha1-Jqcarwc7OfshJxcnRhMcJwQCjbg=",
       "dev": true,
       "requires": {
-        "assign-symbols": "^1.0.0",
-        "is-extendable": "^1.0.1"
+        "assign-symbols": "1.0.0",
+        "is-extendable": "1.0.1"
       },
       "dependencies": {
         "is-extendable": {
@@ -4716,7 +4566,7 @@
           "integrity": "sha512-arnXMxT1hhoKo9k1LZdmlNyJdDDfy2v0fXjFlmok4+i8ul/6WlbVge9bhM74OpNPQPMGUToDtz+KXa1PneJxOA==",
           "dev": true,
           "requires": {
-            "is-plain-object": "^2.0.4"
+            "is-plain-object": "2.0.4"
           }
         }
       }
@@ -4726,8 +4576,9 @@
       "resolved": "https://registry.npmjs.org/extglob/-/extglob-0.3.2.tgz",
       "integrity": "sha1-Lhj/PS9JqydlzskCPwEdqo2DSaE=",
       "dev": true,
+      "optional": true,
       "requires": {
-        "is-extglob": "^1.0.0"
+        "is-extglob": "1.0.0"
       }
     },
     "extract-zip": {
@@ -4784,7 +4635,7 @@
       "integrity": "sha1-i1vL2ewyfFBBv5qwI/1nUPEXfmU=",
       "dev": true,
       "requires": {
-        "pend": "~1.2.0"
+        "pend": "1.2.0"
       }
     },
     "figures": {
@@ -4793,8 +4644,8 @@
       "integrity": "sha1-y+Hjr/zxzUS4DK3+0o3Hk6lwHS4=",
       "dev": true,
       "requires": {
-        "escape-string-regexp": "^1.0.5",
-        "object-assign": "^4.1.0"
+        "escape-string-regexp": "1.0.5",
+        "object-assign": "4.1.1"
       }
     },
     "file-entry-cache": {
@@ -4803,8 +4654,8 @@
       "integrity": "sha1-w5KZDD5oR4PYOLjISkXYoEhFg2E=",
       "dev": true,
       "requires": {
-        "flat-cache": "^1.2.1",
-        "object-assign": "^4.0.1"
+        "flat-cache": "1.3.0",
+        "object-assign": "4.1.1"
       }
     },
     "file-loader": {
@@ -4813,8 +4664,8 @@
       "integrity": "sha512-YCsBfd1ZGCyonOKLxPiKPdu+8ld9HAaMEvJewzz+b2eTF7uL5Zm/HdBF6FjCrpCMRq25Mi0U1gl4pwn2TlH7hQ==",
       "dev": true,
       "requires": {
-        "loader-utils": "^1.0.2",
-        "schema-utils": "^1.0.0"
+        "loader-utils": "1.1.0",
+        "schema-utils": "1.0.0"
       }
     },
     "file-sync-cmp": {
@@ -4827,19 +4678,21 @@
       "version": "2.0.1",
       "resolved": "https://registry.npmjs.org/filename-regex/-/filename-regex-2.0.1.tgz",
       "integrity": "sha1-wcS5vuPglyXdsQa3XB4wH+LxiyY=",
-      "dev": true
+      "dev": true,
+      "optional": true
     },
     "fill-range": {
       "version": "2.2.4",
       "resolved": "https://registry.npmjs.org/fill-range/-/fill-range-2.2.4.tgz",
       "integrity": "sha512-cnrcCbj01+j2gTG921VZPnHbjmdAf8oQV/iGeV2kZxGSyfYjjTyY79ErsK1WJWMpw6DaApEX72binqJE+/d+5Q==",
       "dev": true,
+      "optional": true,
       "requires": {
-        "is-number": "^2.1.0",
-        "isobject": "^2.0.0",
-        "randomatic": "^3.0.0",
-        "repeat-element": "^1.1.2",
-        "repeat-string": "^1.5.2"
+        "is-number": "2.1.0",
+        "isobject": "2.1.0",
+        "randomatic": "3.1.1",
+        "repeat-element": "1.1.3",
+        "repeat-string": "1.6.1"
       }
     },
     "finalhandler": {
@@ -4849,12 +4702,12 @@
       "dev": true,
       "requires": {
         "debug": "2.6.9",
-        "encodeurl": "~1.0.2",
-        "escape-html": "~1.0.3",
-        "on-finished": "~2.3.0",
-        "parseurl": "~1.3.2",
-        "statuses": "~1.4.0",
-        "unpipe": "~1.0.0"
+        "encodeurl": "1.0.2",
+        "escape-html": "1.0.3",
+        "on-finished": "2.3.0",
+        "parseurl": "1.3.2",
+        "statuses": "1.4.0",
+        "unpipe": "1.0.0"
       }
     },
     "find-cache-dir": {
@@ -4863,9 +4716,9 @@
       "integrity": "sha1-kojj6ePMN0hxfTnq3hfPcfww7m8=",
       "dev": true,
       "requires": {
-        "commondir": "^1.0.1",
-        "make-dir": "^1.0.0",
-        "pkg-dir": "^2.0.0"
+        "commondir": "1.0.1",
+        "make-dir": "1.3.0",
+        "pkg-dir": "2.0.0"
       }
     },
     "find-up": {
@@ -4874,7 +4727,7 @@
       "integrity": "sha1-RdG35QbHF93UgndaK3eSCjwMV6c=",
       "dev": true,
       "requires": {
-        "locate-path": "^2.0.0"
+        "locate-path": "2.0.0"
       }
     },
     "findup-sync": {
@@ -4883,8 +4736,8 @@
       "integrity": "sha1-fz56l7gjksZTvwZYm9hRkOk8NoM=",
       "dev": true,
       "requires": {
-        "glob": "~3.2.9",
-        "lodash": "~2.4.1"
+        "glob": "3.2.11",
+        "lodash": "2.4.2"
       },
       "dependencies": {
         "lodash": {
@@ -4901,12 +4754,18 @@
       "integrity": "sha1-0wMLMrOBVPTjt+nHCfSQ9++XxIE=",
       "dev": true,
       "requires": {
-        "circular-json": "^0.3.1",
-        "del": "^2.0.2",
-        "graceful-fs": "^4.1.2",
-        "write": "^0.2.1"
+        "circular-json": "0.3.3",
+        "del": "2.2.2",
+        "graceful-fs": "4.1.15",
+        "write": "0.2.1"
       }
     },
+    "flatted": {
+      "version": "2.0.1",
+      "resolved": "https://registry.npmjs.org/flatted/-/flatted-2.0.1.tgz",
+      "integrity": "sha512-a1hQMktqW9Nmqr5aktAux3JMNqaucxGcjtjWnZLHX7yyPCmlSV3M54nGYbqT8K+0GhF3NBgmJCc3ma+WOgX8Jg==",
+      "dev": true
+    },
     "flatten": {
       "version": "1.0.2",
       "resolved": "https://registry.npmjs.org/flatten/-/flatten-1.0.2.tgz",
@@ -4919,8 +4778,8 @@
       "integrity": "sha512-calZMC10u0FMUqoiunI2AiGIIUtUIvifNwkHhNupZH4cbNnW1Itkoh/Nf5HFYmDrwWPjrUxpkZT0KhuCq0jmGw==",
       "dev": true,
       "requires": {
-        "inherits": "^2.0.1",
-        "readable-stream": "^2.0.4"
+        "inherits": "2.0.3",
+        "readable-stream": "2.3.6"
       },
       "dependencies": {
         "isarray": {
@@ -4935,13 +4794,13 @@
           "integrity": "sha512-tQtKA9WIAhBF3+VLAseyMqZeBjW0AHJoxOtYqSUZNJxauErmLbVm2FW1y+J/YA9dUrAC39ITejlZWhVIwawkKw==",
           "dev": true,
           "requires": {
-            "core-util-is": "~1.0.0",
-            "inherits": "~2.0.3",
-            "isarray": "~1.0.0",
-            "process-nextick-args": "~2.0.0",
-            "safe-buffer": "~5.1.1",
-            "string_decoder": "~1.1.1",
-            "util-deprecate": "~1.0.1"
+            "core-util-is": "1.0.2",
+            "inherits": "2.0.3",
+            "isarray": "1.0.0",
+            "process-nextick-args": "2.0.0",
+            "safe-buffer": "5.1.2",
+            "string_decoder": "1.1.1",
+            "util-deprecate": "1.0.2"
           }
         },
         "string_decoder": {
@@ -4950,7 +4809,7 @@
           "integrity": "sha512-n/ShnvDi6FHbbVfviro+WojiFzv+s8MPMHBczVePfUpDJLwoLT0ht1l4YwBCbi8pJAveEEdnkHyPyTP/mzRfwg==",
           "dev": true,
           "requires": {
-            "safe-buffer": "~5.1.0"
+            "safe-buffer": "5.1.2"
           }
         }
       }
@@ -4961,7 +4820,7 @@
       "integrity": "sha512-Bh65EZI/RU8nx0wbYF9shkFZlqLP+6WT/5FnA3cE/djNSuKNHJEinGGZgu/cQEkeeb2GdFOgenAmn8qaqYke2w==",
       "dev": true,
       "requires": {
-        "debug": "=3.1.0"
+        "debug": "3.1.0"
       },
       "dependencies": {
         "debug": {
@@ -4986,8 +4845,9 @@
       "resolved": "https://registry.npmjs.org/for-own/-/for-own-0.1.5.tgz",
       "integrity": "sha1-UmXGgaTylNq78XyVCbZ2OqhFEM4=",
       "dev": true,
+      "optional": true,
       "requires": {
-        "for-in": "^1.0.1"
+        "for-in": "1.0.2"
       }
     },
     "forever-agent": {
@@ -5002,9 +4862,9 @@
       "integrity": "sha512-1lLKB2Mu3aGP1Q/2eCOx0fNbRMe7XdwktwOruhfqqd0rIJWwN4Dh+E3hrPSlDCXnSR7UtZ1N38rVXm+6+MEhJQ==",
       "dev": true,
       "requires": {
-        "asynckit": "^0.4.0",
-        "combined-stream": "^1.0.6",
-        "mime-types": "^2.1.12"
+        "asynckit": "0.4.0",
+        "combined-stream": "1.0.7",
+        "mime-types": "2.1.21"
       }
     },
     "forwarded": {
@@ -5019,7 +4879,7 @@
       "integrity": "sha1-QpD60n8T6Jvn8zeZxrxaCr//DRk=",
       "dev": true,
       "requires": {
-        "map-cache": "^0.2.2"
+        "map-cache": "0.2.2"
       }
     },
     "fresh": {
@@ -5034,8 +4894,8 @@
       "integrity": "sha1-i/tVAr3kpNNs/e6gB/zKIdfjgq8=",
       "dev": true,
       "requires": {
-        "inherits": "^2.0.1",
-        "readable-stream": "^2.0.0"
+        "inherits": "2.0.3",
+        "readable-stream": "2.3.6"
       },
       "dependencies": {
         "isarray": {
@@ -5050,13 +4910,13 @@
           "integrity": "sha512-tQtKA9WIAhBF3+VLAseyMqZeBjW0AHJoxOtYqSUZNJxauErmLbVm2FW1y+J/YA9dUrAC39ITejlZWhVIwawkKw==",
           "dev": true,
           "requires": {
-            "core-util-is": "~1.0.0",
-            "inherits": "~2.0.3",
-            "isarray": "~1.0.0",
-            "process-nextick-args": "~2.0.0",
-            "safe-buffer": "~5.1.1",
-            "string_decoder": "~1.1.1",
-            "util-deprecate": "~1.0.1"
+            "core-util-is": "1.0.2",
+            "inherits": "2.0.3",
+            "isarray": "1.0.0",
+            "process-nextick-args": "2.0.0",
+            "safe-buffer": "5.1.2",
+            "string_decoder": "1.1.1",
+            "util-deprecate": "1.0.2"
           }
         },
         "string_decoder": {
@@ -5065,7 +4925,7 @@
           "integrity": "sha512-n/ShnvDi6FHbbVfviro+WojiFzv+s8MPMHBczVePfUpDJLwoLT0ht1l4YwBCbi8pJAveEEdnkHyPyTP/mzRfwg==",
           "dev": true,
           "requires": {
-            "safe-buffer": "~5.1.0"
+            "safe-buffer": "5.1.2"
           }
         }
       }
@@ -5076,9 +4936,9 @@
       "integrity": "sha1-zTzl9+fLYUWIP8rjGR6Yd/hYeVA=",
       "dev": true,
       "requires": {
-        "graceful-fs": "^4.1.2",
-        "jsonfile": "^2.1.0",
-        "klaw": "^1.0.0"
+        "graceful-fs": "4.1.15",
+        "jsonfile": "2.4.0",
+        "klaw": "1.3.1"
       }
     },
     "fs-readdir-recursive": {
@@ -5093,10 +4953,10 @@
       "integrity": "sha1-tH31NJPvkR33VzHnCp3tAYnbQMk=",
       "dev": true,
       "requires": {
-        "graceful-fs": "^4.1.2",
-        "iferr": "^0.1.5",
-        "imurmurhash": "^0.1.4",
-        "readable-stream": "1 || 2"
+        "graceful-fs": "4.1.15",
+        "iferr": "0.1.5",
+        "imurmurhash": "0.1.4",
+        "readable-stream": "1.0.34"
       }
     },
     "fs.realpath": {
@@ -5105,2115 +4965,1580 @@
       "integrity": "sha1-FQStJSMVjKpA20onh8sBQRmU6k8=",
       "dev": true
     },
-    "fsevents": {
-      "version": "1.2.4",
-      "resolved": "https://registry.npmjs.org/fsevents/-/fsevents-1.2.4.tgz",
-      "integrity": "sha512-z8H8/diyk76B7q5wg+Ud0+CqzcAF3mBBI/bA5ne5zrRUUIvNkJY//D3BqyH571KuAC4Nr7Rw7CjWX4r0y9DvNg==",
+    "function-bind": {
+      "version": "1.1.1",
+      "resolved": "https://registry.npmjs.org/function-bind/-/function-bind-1.1.1.tgz",
+      "integrity": "sha512-yIovAzMX49sF8Yl58fSCWJ5svSLuaibPxXQJFLmBObTuCr0Mf1KiPopGM9NiFjiYBCbfaa2Fh6breQ6ANVTI0A==",
+      "dev": true
+    },
+    "gaze": {
+      "version": "0.5.2",
+      "resolved": "https://registry.npmjs.org/gaze/-/gaze-0.5.2.tgz",
+      "integrity": "sha1-QLcJU30k0dRXZ9takIaJ3+aaxE8=",
+      "dev": true,
+      "requires": {
+        "globule": "0.1.0"
+      }
+    },
+    "generate-function": {
+      "version": "2.3.1",
+      "resolved": "https://registry.npmjs.org/generate-function/-/generate-function-2.3.1.tgz",
+      "integrity": "sha512-eeB5GfMNeevm/GRYq20ShmsaGcmI81kIX2K9XQx5miC8KdHaC6Jm0qQ8ZNeGOi7wYB8OsdxKs+Y2oVuTFuVwKQ==",
+      "dev": true,
+      "requires": {
+        "is-property": "1.0.2"
+      }
+    },
+    "generate-object-property": {
+      "version": "1.2.0",
+      "resolved": "https://registry.npmjs.org/generate-object-property/-/generate-object-property-1.2.0.tgz",
+      "integrity": "sha1-nA4cQDCM6AT0eDYYuTf6iPmdUNA=",
+      "dev": true,
+      "requires": {
+        "is-property": "1.0.2"
+      }
+    },
+    "get-caller-file": {
+      "version": "1.0.3",
+      "resolved": "https://registry.npmjs.org/get-caller-file/-/get-caller-file-1.0.3.tgz",
+      "integrity": "sha512-3t6rVToeoZfYSGd8YoLFR2DJkiQrIiUrGcjvFX2mDw3bn6k2OtwHN0TNCLbBO+w8qTvimhDkv+LSscbJY1vE6w==",
+      "dev": true
+    },
+    "get-stdin": {
+      "version": "4.0.1",
+      "resolved": "https://registry.npmjs.org/get-stdin/-/get-stdin-4.0.1.tgz",
+      "integrity": "sha1-uWjGsKBDhDJJAui/Gl3zJXmkUP4=",
+      "dev": true
+    },
+    "get-stream": {
+      "version": "3.0.0",
+      "resolved": "https://registry.npmjs.org/get-stream/-/get-stream-3.0.0.tgz",
+      "integrity": "sha1-jpQ9E1jcN1VQVOy+LtsFqhdO3hQ=",
+      "dev": true
+    },
+    "get-value": {
+      "version": "2.0.6",
+      "resolved": "https://registry.npmjs.org/get-value/-/get-value-2.0.6.tgz",
+      "integrity": "sha1-3BXKHGcjh8p2vTesCjlbogQqLCg=",
+      "dev": true
+    },
+    "getobject": {
+      "version": "0.1.0",
+      "resolved": "https://registry.npmjs.org/getobject/-/getobject-0.1.0.tgz",
+      "integrity": "sha1-BHpEl4n6Fg0Bj1SG7ZEyC27HiFw=",
+      "dev": true
+    },
+    "getpass": {
+      "version": "0.1.7",
+      "resolved": "https://registry.npmjs.org/getpass/-/getpass-0.1.7.tgz",
+      "integrity": "sha1-Xv+OPmhNVprkyysSgmBOi6YhSfo=",
+      "dev": true,
+      "requires": {
+        "assert-plus": "1.0.0"
+      }
+    },
+    "github-markdown-css": {
+      "version": "2.6.0",
+      "resolved": "https://registry.npmjs.org/github-markdown-css/-/github-markdown-css-2.6.0.tgz",
+      "integrity": "sha1-zcdLq1ZrA51/u3RgH3ghsQnPWRg="
+    },
+    "glob": {
+      "version": "3.2.11",
+      "resolved": "https://registry.npmjs.org/glob/-/glob-3.2.11.tgz",
+      "integrity": "sha1-Spc/Y1uRkPcV0QmH1cAP0oFevj0=",
+      "requires": {
+        "inherits": "2.0.3",
+        "minimatch": "0.3.0"
+      }
+    },
+    "glob-base": {
+      "version": "0.3.0",
+      "resolved": "https://registry.npmjs.org/glob-base/-/glob-base-0.3.0.tgz",
+      "integrity": "sha1-27Fk9iIbHAscz4Kuoyi0l98Oo8Q=",
+      "dev": true,
+      "requires": {
+        "glob-parent": "2.0.0",
+        "is-glob": "2.0.1"
+      }
+    },
+    "glob-parent": {
+      "version": "2.0.0",
+      "resolved": "https://registry.npmjs.org/glob-parent/-/glob-parent-2.0.0.tgz",
+      "integrity": "sha1-gTg9ctsFT8zPUzbaqQLxgvbtuyg=",
+      "dev": true,
+      "requires": {
+        "is-glob": "2.0.1"
+      }
+    },
+    "global-modules-path": {
+      "version": "2.3.0",
+      "resolved": "https://registry.npmjs.org/global-modules-path/-/global-modules-path-2.3.0.tgz",
+      "integrity": "sha512-HchvMJNYh9dGSCy8pOQ2O8u/hoXaL+0XhnrwH0RyLiSXMMTl9W3N6KUU73+JFOg5PGjtzl6VZzUQsnrpm7Szag==",
+      "dev": true
+    },
+    "globals": {
+      "version": "9.18.0",
+      "resolved": "https://registry.npmjs.org/globals/-/globals-9.18.0.tgz",
+      "integrity": "sha512-S0nG3CLEQiY/ILxqtztTWH/3iRRdyBLw6KMDxnKMchrtbj2OFmehVh0WUCfW3DUrIgx/qFrJPICrq4Z4sTR9UQ==",
+      "dev": true
+    },
+    "globby": {
+      "version": "7.1.1",
+      "resolved": "https://registry.npmjs.org/globby/-/globby-7.1.1.tgz",
+      "integrity": "sha1-+yzP+UAfhgCUXfral0QMypcrhoA=",
       "dev": true,
-      "optional": true,
       "requires": {
-        "nan": "^2.9.2",
-        "node-pre-gyp": "^0.10.0"
+        "array-union": "1.0.2",
+        "dir-glob": "2.0.0",
+        "glob": "7.1.3",
+        "ignore": "3.3.10",
+        "pify": "3.0.0",
+        "slash": "1.0.0"
       },
       "dependencies": {
-        "abbrev": {
-          "version": "1.1.1",
-          "bundled": true,
+        "glob": {
+          "version": "7.1.3",
+          "resolved": "https://registry.npmjs.org/glob/-/glob-7.1.3.tgz",
+          "integrity": "sha512-vcfuiIxogLV4DlGBHIUOwI0IbrJ8HWPc4MU7HzviGeNho/UJDfi6B5p3sHeWIQ0KGIU0Jpxi5ZHxemQfLkkAwQ==",
           "dev": true,
-          "optional": true
-        },
-        "ansi-regex": {
-          "version": "2.1.1",
-          "bundled": true,
-          "dev": true
+          "requires": {
+            "fs.realpath": "1.0.0",
+            "inflight": "1.0.6",
+            "inherits": "2.0.3",
+            "minimatch": "3.0.4",
+            "once": "1.4.0",
+            "path-is-absolute": "1.0.1"
+          }
         },
-        "aproba": {
-          "version": "1.2.0",
-          "bundled": true,
-          "dev": true,
-          "optional": true
-        },
-        "are-we-there-yet": {
-          "version": "1.1.4",
-          "bundled": true,
+        "minimatch": {
+          "version": "3.0.4",
+          "resolved": "https://registry.npmjs.org/minimatch/-/minimatch-3.0.4.tgz",
+          "integrity": "sha512-yJHVQEhyqPLUTgt9B83PXu6W3rx4MvvHvSUvToogpwoGDOUQ+yDrR0HRot+yOCdCO7u4hX3pWft6kWBBcqh0UA==",
           "dev": true,
-          "optional": true,
           "requires": {
-            "delegates": "^1.0.0",
-            "readable-stream": "^2.0.6"
+            "brace-expansion": "1.1.11"
           }
-        },
-        "balanced-match": {
-          "version": "1.0.0",
-          "bundled": true,
-          "dev": true,
-          "optional": true
-        },
-        "brace-expansion": {
-          "version": "1.1.11",
-          "bundled": true,
+        }
+      }
+    },
+    "globule": {
+      "version": "0.1.0",
+      "resolved": "https://registry.npmjs.org/globule/-/globule-0.1.0.tgz",
+      "integrity": "sha1-2cjt3h2nnRJaFRt5UzuXhnY0auU=",
+      "dev": true,
+      "requires": {
+        "glob": "3.1.21",
+        "lodash": "1.0.2",
+        "minimatch": "0.2.14"
+      },
+      "dependencies": {
+        "glob": {
+          "version": "3.1.21",
+          "resolved": "https://registry.npmjs.org/glob/-/glob-3.1.21.tgz",
+          "integrity": "sha1-0p4KBV3qUTj00H7UDomC6DwgZs0=",
           "dev": true,
-          "optional": true,
           "requires": {
-            "balanced-match": "^1.0.0",
-            "concat-map": "0.0.1"
+            "graceful-fs": "1.2.3",
+            "inherits": "1.0.2",
+            "minimatch": "0.2.14"
           }
         },
-        "chownr": {
-          "version": "1.0.1",
-          "bundled": true,
-          "dev": true,
-          "optional": true
-        },
-        "code-point-at": {
-          "version": "1.1.0",
-          "bundled": true,
-          "dev": true,
-          "optional": true
-        },
-        "concat-map": {
-          "version": "0.0.1",
-          "bundled": true,
-          "dev": true,
-          "optional": true
+        "graceful-fs": {
+          "version": "1.2.3",
+          "resolved": "https://registry.npmjs.org/graceful-fs/-/graceful-fs-1.2.3.tgz",
+          "integrity": "sha1-FaSAaldUfLLS2/J/QuiajDRRs2Q=",
+          "dev": true
         },
-        "console-control-strings": {
-          "version": "1.1.0",
-          "bundled": true,
-          "dev": true,
-          "optional": true
+        "inherits": {
+          "version": "1.0.2",
+          "resolved": "https://registry.npmjs.org/inherits/-/inherits-1.0.2.tgz",
+          "integrity": "sha1-ykMJ2t7mtUzAuNJH6NfHoJdb3Js=",
+          "dev": true
         },
-        "core-util-is": {
+        "lodash": {
           "version": "1.0.2",
-          "bundled": true,
-          "dev": true,
-          "optional": true
+          "resolved": "https://registry.npmjs.org/lodash/-/lodash-1.0.2.tgz",
+          "integrity": "sha1-j1dWDIO1n8JwvT1WG2kAQ0MOJVE=",
+          "dev": true
         },
-        "debug": {
-          "version": "2.6.9",
-          "bundled": true,
+        "minimatch": {
+          "version": "0.2.14",
+          "resolved": "https://registry.npmjs.org/minimatch/-/minimatch-0.2.14.tgz",
+          "integrity": "sha1-x054BXT2PG+aCQ6Q775u9TpqdWo=",
           "dev": true,
-          "optional": true,
           "requires": {
-            "ms": "2.0.0"
+            "lru-cache": "2.7.3",
+            "sigmund": "1.0.1"
           }
-        },
-        "deep-extend": {
-          "version": "0.5.1",
-          "bundled": true,
-          "dev": true,
-          "optional": true
-        },
-        "delegates": {
-          "version": "1.0.0",
-          "bundled": true,
-          "dev": true,
-          "optional": true
-        },
-        "detect-libc": {
-          "version": "1.0.3",
-          "bundled": true,
-          "dev": true,
-          "optional": true
-        },
-        "fs-minipass": {
-          "version": "1.2.5",
-          "bundled": true,
+        }
+      }
+    },
+    "graceful-fs": {
+      "version": "4.1.15",
+      "resolved": "https://registry.npmjs.org/graceful-fs/-/graceful-fs-4.1.15.tgz",
+      "integrity": "sha512-6uHUhOPEBgQ24HM+r6b/QwWfZq+yiFcipKFrOFiBEnWdy5sdzYoi+pJeQaPI5qOLRFqWmAXUPQNsielzdLoecA==",
+      "dev": true
+    },
+    "grunt": {
+      "version": "0.4.5",
+      "resolved": "https://registry.npmjs.org/grunt/-/grunt-0.4.5.tgz",
+      "integrity": "sha1-VpN81RlDJK3/bSB2MYMqnWuk5/A=",
+      "dev": true,
+      "requires": {
+        "async": "0.1.22",
+        "coffee-script": "1.3.3",
+        "colors": "0.6.2",
+        "dateformat": "1.0.2-1.2.3",
+        "eventemitter2": "0.4.14",
+        "exit": "0.1.2",
+        "findup-sync": "0.1.3",
+        "getobject": "0.1.0",
+        "glob": "3.1.21",
+        "grunt-legacy-log": "0.1.3",
+        "grunt-legacy-util": "0.2.0",
+        "hooker": "0.2.3",
+        "iconv-lite": "0.2.11",
+        "js-yaml": "2.0.5",
+        "lodash": "0.9.2",
+        "minimatch": "0.2.14",
+        "nopt": "1.0.10",
+        "rimraf": "2.2.8",
+        "underscore.string": "2.2.1",
+        "which": "1.0.9"
+      },
+      "dependencies": {
+        "argparse": {
+          "version": "0.1.16",
+          "resolved": "https://registry.npmjs.org/argparse/-/argparse-0.1.16.tgz",
+          "integrity": "sha1-z9AeD7uj1srtBJ+9dY1A9lGW9Xw=",
           "dev": true,
-          "optional": true,
           "requires": {
-            "minipass": "^2.2.1"
+            "underscore": "1.7.0",
+            "underscore.string": "2.4.0"
+          },
+          "dependencies": {
+            "underscore.string": {
+              "version": "2.4.0",
+              "resolved": "https://registry.npmjs.org/underscore.string/-/underscore.string-2.4.0.tgz",
+              "integrity": "sha1-jN2PusTi0uoefi6Al8QvRCKA+Fs=",
+              "dev": true
+            }
           }
         },
-        "fs.realpath": {
-          "version": "1.0.0",
-          "bundled": true,
-          "dev": true,
-          "optional": true
+        "async": {
+          "version": "0.1.22",
+          "resolved": "https://registry.npmjs.org/async/-/async-0.1.22.tgz",
+          "integrity": "sha1-D8GqoIig4+8Ovi2IMbqw3PiEUGE=",
+          "dev": true
         },
-        "gauge": {
-          "version": "2.7.4",
-          "bundled": true,
-          "dev": true,
-          "optional": true,
-          "requires": {
-            "aproba": "^1.0.3",
-            "console-control-strings": "^1.0.0",
-            "has-unicode": "^2.0.0",
-            "object-assign": "^4.1.0",
-            "signal-exit": "^3.0.0",
-            "string-width": "^1.0.1",
-            "strip-ansi": "^3.0.1",
-            "wide-align": "^1.1.0"
-          }
+        "colors": {
+          "version": "0.6.2",
+          "resolved": "https://registry.npmjs.org/colors/-/colors-0.6.2.tgz",
+          "integrity": "sha1-JCP+ZnisDF2uiFLl0OW+CMmXq8w=",
+          "dev": true
+        },
+        "esprima": {
+          "version": "1.0.4",
+          "resolved": "https://registry.npmjs.org/esprima/-/esprima-1.0.4.tgz",
+          "integrity": "sha1-n1V+CPw7TSbs6d00+Pv0drYlha0=",
+          "dev": true
         },
         "glob": {
-          "version": "7.1.2",
-          "bundled": true,
+          "version": "3.1.21",
+          "resolved": "https://registry.npmjs.org/glob/-/glob-3.1.21.tgz",
+          "integrity": "sha1-0p4KBV3qUTj00H7UDomC6DwgZs0=",
           "dev": true,
-          "optional": true,
           "requires": {
-            "fs.realpath": "^1.0.0",
-            "inflight": "^1.0.4",
-            "inherits": "2",
-            "minimatch": "^3.0.4",
-            "once": "^1.3.0",
-            "path-is-absolute": "^1.0.0"
+            "graceful-fs": "1.2.3",
+            "inherits": "1.0.2",
+            "minimatch": "0.2.14"
           }
         },
-        "has-unicode": {
-          "version": "2.0.1",
-          "bundled": true,
-          "dev": true,
-          "optional": true
+        "graceful-fs": {
+          "version": "1.2.3",
+          "resolved": "https://registry.npmjs.org/graceful-fs/-/graceful-fs-1.2.3.tgz",
+          "integrity": "sha1-FaSAaldUfLLS2/J/QuiajDRRs2Q=",
+          "dev": true
         },
         "iconv-lite": {
-          "version": "0.4.21",
-          "bundled": true,
-          "dev": true,
-          "optional": true,
-          "requires": {
-            "safer-buffer": "^2.1.0"
-          }
+          "version": "0.2.11",
+          "resolved": "https://registry.npmjs.org/iconv-lite/-/iconv-lite-0.2.11.tgz",
+          "integrity": "sha1-HOYKOleGSiktEyH/RgnKS7llrcg=",
+          "dev": true
         },
-        "ignore-walk": {
-          "version": "3.0.1",
-          "bundled": true,
+        "inherits": {
+          "version": "1.0.2",
+          "resolved": "https://registry.npmjs.org/inherits/-/inherits-1.0.2.tgz",
+          "integrity": "sha1-ykMJ2t7mtUzAuNJH6NfHoJdb3Js=",
+          "dev": true
+        },
+        "js-yaml": {
+          "version": "2.0.5",
+          "resolved": "https://registry.npmjs.org/js-yaml/-/js-yaml-2.0.5.tgz",
+          "integrity": "sha1-olrmUJmZ6X3yeMZxnaEb0Gh3Q6g=",
           "dev": true,
-          "optional": true,
           "requires": {
-            "minimatch": "^3.0.4"
+            "argparse": "0.1.16",
+            "esprima": "1.0.4"
           }
         },
-        "inflight": {
-          "version": "1.0.6",
-          "bundled": true,
+        "lodash": {
+          "version": "0.9.2",
+          "resolved": "https://registry.npmjs.org/lodash/-/lodash-0.9.2.tgz",
+          "integrity": "sha1-jzSZxSRdNG1oLlsNO0B2fgnxqSw=",
+          "dev": true
+        },
+        "minimatch": {
+          "version": "0.2.14",
+          "resolved": "https://registry.npmjs.org/minimatch/-/minimatch-0.2.14.tgz",
+          "integrity": "sha1-x054BXT2PG+aCQ6Q775u9TpqdWo=",
           "dev": true,
-          "optional": true,
           "requires": {
-            "once": "^1.3.0",
-            "wrappy": "1"
+            "lru-cache": "2.7.3",
+            "sigmund": "1.0.1"
           }
         },
-        "inherits": {
-          "version": "2.0.3",
-          "bundled": true,
-          "dev": true,
-          "optional": true
-        },
-        "ini": {
-          "version": "1.3.5",
-          "bundled": true,
-          "dev": true,
-          "optional": true
+        "rimraf": {
+          "version": "2.2.8",
+          "resolved": "https://registry.npmjs.org/rimraf/-/rimraf-2.2.8.tgz",
+          "integrity": "sha1-5Dm+Kq7jJzIZUnMPmaiSnk/FBYI=",
+          "dev": true
         },
-        "is-fullwidth-code-point": {
+        "underscore": {
+          "version": "1.7.0",
+          "resolved": "https://registry.npmjs.org/underscore/-/underscore-1.7.0.tgz",
+          "integrity": "sha1-a7rwh3UA02vjTsqlhODbn+8DUgk=",
+          "dev": true
+        }
+      }
+    },
+    "grunt-angular-templates": {
+      "version": "0.5.9",
+      "resolved": "https://registry.npmjs.org/grunt-angular-templates/-/grunt-angular-templates-0.5.9.tgz",
+      "integrity": "sha1-KJm+INlDitGbDQqAaqjseiOyWyo=",
+      "requires": {
+        "html-minifier": "0.6.9"
+      }
+    },
+    "grunt-cache-bust": {
+      "version": "1.3.0",
+      "resolved": "https://registry.npmjs.org/grunt-cache-bust/-/grunt-cache-bust-1.3.0.tgz",
+      "integrity": "sha1-YtkgjiMV8cIMFgg6kHzkq8JJv1Q=",
+      "dev": true
+    },
+    "grunt-cli": {
+      "version": "0.1.13",
+      "resolved": "https://registry.npmjs.org/grunt-cli/-/grunt-cli-0.1.13.tgz",
+      "integrity": "sha1-6evEBHYx9QEtkidww5N4EzytEPQ=",
+      "dev": true,
+      "requires": {
+        "findup-sync": "0.1.3",
+        "nopt": "1.0.10",
+        "resolve": "0.3.1"
+      },
+      "dependencies": {
+        "resolve": {
+          "version": "0.3.1",
+          "resolved": "https://registry.npmjs.org/resolve/-/resolve-0.3.1.tgz",
+          "integrity": "sha1-NMY0R8ZkxwWY0cmxJvxDsqJDEKQ=",
+          "dev": true
+        }
+      }
+    },
+    "grunt-concurrent": {
+      "version": "0.5.0",
+      "resolved": "https://registry.npmjs.org/grunt-concurrent/-/grunt-concurrent-0.5.0.tgz",
+      "integrity": "sha1-SlGaTCh4JfDeBxX3O4XRUMdQ2fc=",
+      "dev": true,
+      "requires": {
+        "async": "0.2.10",
+        "pad-stdio": "0.1.1"
+      }
+    },
+    "grunt-contrib-concat": {
+      "version": "0.4.0",
+      "resolved": "https://registry.npmjs.org/grunt-contrib-concat/-/grunt-contrib-concat-0.4.0.tgz",
+      "integrity": "sha1-uH988VO/ZGiBQvlHFhFWAT+8fHQ=",
+      "dev": true,
+      "requires": {
+        "chalk": "0.4.0"
+      },
+      "dependencies": {
+        "ansi-styles": {
           "version": "1.0.0",
-          "bundled": true,
+          "resolved": "https://registry.npmjs.org/ansi-styles/-/ansi-styles-1.0.0.tgz",
+          "integrity": "sha1-yxAt8cVvUSPquLZ817mAJ6AnkXg=",
+          "dev": true
+        },
+        "chalk": {
+          "version": "0.4.0",
+          "resolved": "https://registry.npmjs.org/chalk/-/chalk-0.4.0.tgz",
+          "integrity": "sha1-UZmj3c0MHv4jvAjBsCewYXbgxk8=",
           "dev": true,
-          "optional": true,
           "requires": {
-            "number-is-nan": "^1.0.0"
+            "ansi-styles": "1.0.0",
+            "has-color": "0.1.7",
+            "strip-ansi": "0.1.1"
           }
         },
-        "isarray": {
-          "version": "1.0.0",
-          "bundled": true,
-          "dev": true,
-          "optional": true
-        },
-        "minimatch": {
-          "version": "3.0.4",
-          "bundled": true,
-          "dev": true,
-          "optional": true,
-          "requires": {
-            "brace-expansion": "^1.1.7"
-          }
-        },
-        "minimist": {
-          "version": "0.0.8",
-          "bundled": true,
-          "dev": true,
-          "optional": true
+        "strip-ansi": {
+          "version": "0.1.1",
+          "resolved": "https://registry.npmjs.org/strip-ansi/-/strip-ansi-0.1.1.tgz",
+          "integrity": "sha1-OeipjQRNFQZgq+SmgIrPcLt7yZE=",
+          "dev": true
+        }
+      }
+    },
+    "grunt-contrib-copy": {
+      "version": "0.5.0",
+      "resolved": "https://registry.npmjs.org/grunt-contrib-copy/-/grunt-contrib-copy-0.5.0.tgz",
+      "integrity": "sha1-QQB1rEWlhWuhkbHMclclRQ1KAhU=",
+      "dev": true
+    },
+    "grunt-contrib-cssmin": {
+      "version": "0.9.0",
+      "resolved": "https://registry.npmjs.org/grunt-contrib-cssmin/-/grunt-contrib-cssmin-0.9.0.tgz",
+      "integrity": "sha1-JyQfAWCohmZZ2rQNyMJ3bAHsfOI=",
+      "dev": true,
+      "requires": {
+        "chalk": "0.4.0",
+        "clean-css": "2.1.8",
+        "maxmin": "0.1.0"
+      },
+      "dependencies": {
+        "ansi-styles": {
+          "version": "1.0.0",
+          "resolved": "https://registry.npmjs.org/ansi-styles/-/ansi-styles-1.0.0.tgz",
+          "integrity": "sha1-yxAt8cVvUSPquLZ817mAJ6AnkXg=",
+          "dev": true
         },
-        "minipass": {
-          "version": "2.2.4",
-          "bundled": true,
+        "chalk": {
+          "version": "0.4.0",
+          "resolved": "https://registry.npmjs.org/chalk/-/chalk-0.4.0.tgz",
+          "integrity": "sha1-UZmj3c0MHv4jvAjBsCewYXbgxk8=",
           "dev": true,
-          "optional": true,
           "requires": {
-            "safe-buffer": "^5.1.1",
-            "yallist": "^3.0.0"
+            "ansi-styles": "1.0.0",
+            "has-color": "0.1.7",
+            "strip-ansi": "0.1.1"
           }
         },
-        "minizlib": {
-          "version": "1.1.0",
-          "bundled": true,
+        "clean-css": {
+          "version": "2.1.8",
+          "resolved": "https://registry.npmjs.org/clean-css/-/clean-css-2.1.8.tgz",
+          "integrity": "sha1-K0sv1g8yRBCWIWriWiH6p0WA3IM=",
           "dev": true,
-          "optional": true,
           "requires": {
-            "minipass": "^2.2.1"
+            "commander": "2.1.0"
           }
         },
-        "mkdirp": {
-          "version": "0.5.1",
-          "bundled": true,
-          "dev": true,
-          "optional": true,
-          "requires": {
-            "minimist": "0.0.8"
-          }
+        "commander": {
+          "version": "2.1.0",
+          "resolved": "https://registry.npmjs.org/commander/-/commander-2.1.0.tgz",
+          "integrity": "sha1-0SG7roYNmZKj1Re6lvVliOR8Z4E=",
+          "dev": true
         },
-        "ms": {
-          "version": "2.0.0",
-          "bundled": true,
-          "dev": true,
-          "optional": true
+        "strip-ansi": {
+          "version": "0.1.1",
+          "resolved": "https://registry.npmjs.org/strip-ansi/-/strip-ansi-0.1.1.tgz",
+          "integrity": "sha1-OeipjQRNFQZgq+SmgIrPcLt7yZE=",
+          "dev": true
+        }
+      }
+    },
+    "grunt-contrib-htmlmin": {
+      "version": "0.3.0",
+      "resolved": "https://registry.npmjs.org/grunt-contrib-htmlmin/-/grunt-contrib-htmlmin-0.3.0.tgz",
+      "integrity": "sha1-yWCAIEj2CZJenQ7xsGcJBLTFo/0=",
+      "dev": true,
+      "requires": {
+        "chalk": "0.4.0",
+        "html-minifier": "0.6.9",
+        "pretty-bytes": "0.1.2"
+      },
+      "dependencies": {
+        "ansi-styles": {
+          "version": "1.0.0",
+          "resolved": "https://registry.npmjs.org/ansi-styles/-/ansi-styles-1.0.0.tgz",
+          "integrity": "sha1-yxAt8cVvUSPquLZ817mAJ6AnkXg=",
+          "dev": true
         },
-        "needle": {
-          "version": "2.2.0",
-          "bundled": true,
+        "chalk": {
+          "version": "0.4.0",
+          "resolved": "https://registry.npmjs.org/chalk/-/chalk-0.4.0.tgz",
+          "integrity": "sha1-UZmj3c0MHv4jvAjBsCewYXbgxk8=",
           "dev": true,
-          "optional": true,
           "requires": {
-            "debug": "^2.1.2",
-            "iconv-lite": "^0.4.4",
-            "sax": "^1.2.4"
+            "ansi-styles": "1.0.0",
+            "has-color": "0.1.7",
+            "strip-ansi": "0.1.1"
           }
         },
-        "node-pre-gyp": {
-          "version": "0.10.0",
-          "bundled": true,
-          "dev": true,
-          "optional": true,
-          "requires": {
-            "detect-libc": "^1.0.2",
-            "mkdirp": "^0.5.1",
-            "needle": "^2.2.0",
-            "nopt": "^4.0.1",
-            "npm-packlist": "^1.1.6",
-            "npmlog": "^4.0.2",
-            "rc": "^1.1.7",
-            "rimraf": "^2.6.1",
-            "semver": "^5.3.0",
-            "tar": "^4"
-          }
+        "strip-ansi": {
+          "version": "0.1.1",
+          "resolved": "https://registry.npmjs.org/strip-ansi/-/strip-ansi-0.1.1.tgz",
+          "integrity": "sha1-OeipjQRNFQZgq+SmgIrPcLt7yZE=",
+          "dev": true
+        }
+      }
+    },
+    "grunt-contrib-uglify": {
+      "version": "0.4.1",
+      "resolved": "https://registry.npmjs.org/grunt-contrib-uglify/-/grunt-contrib-uglify-0.4.1.tgz",
+      "integrity": "sha1-1D87xuAsM1Vj+MT58IE/tLD/ebE=",
+      "dev": true,
+      "requires": {
+        "chalk": "0.4.0",
+        "maxmin": "0.1.0",
+        "uglify-js": "2.4.24"
+      },
+      "dependencies": {
+        "ansi-styles": {
+          "version": "1.0.0",
+          "resolved": "https://registry.npmjs.org/ansi-styles/-/ansi-styles-1.0.0.tgz",
+          "integrity": "sha1-yxAt8cVvUSPquLZ817mAJ6AnkXg=",
+          "dev": true
         },
-        "nopt": {
-          "version": "4.0.1",
-          "bundled": true,
+        "chalk": {
+          "version": "0.4.0",
+          "resolved": "https://registry.npmjs.org/chalk/-/chalk-0.4.0.tgz",
+          "integrity": "sha1-UZmj3c0MHv4jvAjBsCewYXbgxk8=",
           "dev": true,
-          "optional": true,
           "requires": {
-            "abbrev": "1",
-            "osenv": "^0.1.4"
+            "ansi-styles": "1.0.0",
+            "has-color": "0.1.7",
+            "strip-ansi": "0.1.1"
           }
         },
-        "npm-bundled": {
-          "version": "1.0.3",
-          "bundled": true,
-          "dev": true,
-          "optional": true
-        },
-        "npm-packlist": {
-          "version": "1.1.10",
-          "bundled": true,
-          "dev": true,
-          "optional": true,
-          "requires": {
-            "ignore-walk": "^3.0.1",
-            "npm-bundled": "^1.0.1"
-          }
+        "strip-ansi": {
+          "version": "0.1.1",
+          "resolved": "https://registry.npmjs.org/strip-ansi/-/strip-ansi-0.1.1.tgz",
+          "integrity": "sha1-OeipjQRNFQZgq+SmgIrPcLt7yZE=",
+          "dev": true
+        }
+      }
+    },
+    "grunt-contrib-watch": {
+      "version": "0.6.1",
+      "resolved": "https://registry.npmjs.org/grunt-contrib-watch/-/grunt-contrib-watch-0.6.1.tgz",
+      "integrity": "sha1-ZP3LolpjX1tNobbOb5DaCutuPxU=",
+      "dev": true,
+      "requires": {
+        "async": "0.2.10",
+        "gaze": "0.5.2",
+        "lodash": "2.4.2",
+        "tiny-lr-fork": "0.0.5"
+      },
+      "dependencies": {
+        "lodash": {
+          "version": "2.4.2",
+          "resolved": "https://registry.npmjs.org/lodash/-/lodash-2.4.2.tgz",
+          "integrity": "sha1-+t2DS5aDBz2hebPq5tnA0VBT9z4=",
+          "dev": true
+        }
+      }
+    },
+    "grunt-dom-munger": {
+      "version": "3.4.0",
+      "resolved": "https://registry.npmjs.org/grunt-dom-munger/-/grunt-dom-munger-3.4.0.tgz",
+      "integrity": "sha1-LQ2Plk9amVEekUrR1T8fccWrbYk=",
+      "requires": {
+        "cheerio": "0.12.4"
+      }
+    },
+    "grunt-filerev": {
+      "version": "0.2.1",
+      "resolved": "https://registry.npmjs.org/grunt-filerev/-/grunt-filerev-0.2.1.tgz",
+      "integrity": "sha1-Svngz+2nuwFnB2VpeREimBH29NM=",
+      "dev": true,
+      "requires": {
+        "chalk": "0.4.0",
+        "each-async": "0.1.3"
+      },
+      "dependencies": {
+        "ansi-styles": {
+          "version": "1.0.0",
+          "resolved": "https://registry.npmjs.org/ansi-styles/-/ansi-styles-1.0.0.tgz",
+          "integrity": "sha1-yxAt8cVvUSPquLZ817mAJ6AnkXg=",
+          "dev": true
         },
-        "npmlog": {
-          "version": "4.1.2",
-          "bundled": true,
+        "chalk": {
+          "version": "0.4.0",
+          "resolved": "https://registry.npmjs.org/chalk/-/chalk-0.4.0.tgz",
+          "integrity": "sha1-UZmj3c0MHv4jvAjBsCewYXbgxk8=",
           "dev": true,
-          "optional": true,
           "requires": {
-            "are-we-there-yet": "~1.1.2",
-            "console-control-strings": "~1.1.0",
-            "gauge": "~2.7.3",
-            "set-blocking": "~2.0.0"
+            "ansi-styles": "1.0.0",
+            "has-color": "0.1.7",
+            "strip-ansi": "0.1.1"
           }
         },
-        "number-is-nan": {
-          "version": "1.0.1",
-          "bundled": true,
-          "dev": true,
-          "optional": true
+        "strip-ansi": {
+          "version": "0.1.1",
+          "resolved": "https://registry.npmjs.org/strip-ansi/-/strip-ansi-0.1.1.tgz",
+          "integrity": "sha1-OeipjQRNFQZgq+SmgIrPcLt7yZE=",
+          "dev": true
+        }
+      }
+    },
+    "grunt-htmlhint": {
+      "version": "0.9.13",
+      "resolved": "https://registry.npmjs.org/grunt-htmlhint/-/grunt-htmlhint-0.9.13.tgz",
+      "integrity": "sha1-cXACPzDi5wUnkjQrSNW7+RK512w=",
+      "dev": true,
+      "requires": {
+        "htmlhint": "0.9.13"
+      }
+    },
+    "grunt-legacy-log": {
+      "version": "0.1.3",
+      "resolved": "https://registry.npmjs.org/grunt-legacy-log/-/grunt-legacy-log-0.1.3.tgz",
+      "integrity": "sha1-7ClCboAwIa9ZAp+H0vnNczWgVTE=",
+      "dev": true,
+      "requires": {
+        "colors": "0.6.2",
+        "grunt-legacy-log-utils": "0.1.1",
+        "hooker": "0.2.3",
+        "lodash": "2.4.2",
+        "underscore.string": "2.3.3"
+      },
+      "dependencies": {
+        "colors": {
+          "version": "0.6.2",
+          "resolved": "https://registry.npmjs.org/colors/-/colors-0.6.2.tgz",
+          "integrity": "sha1-JCP+ZnisDF2uiFLl0OW+CMmXq8w=",
+          "dev": true
         },
-        "object-assign": {
-          "version": "4.1.1",
-          "bundled": true,
-          "dev": true,
-          "optional": true
+        "lodash": {
+          "version": "2.4.2",
+          "resolved": "https://registry.npmjs.org/lodash/-/lodash-2.4.2.tgz",
+          "integrity": "sha1-+t2DS5aDBz2hebPq5tnA0VBT9z4=",
+          "dev": true
         },
-        "once": {
-          "version": "1.4.0",
-          "bundled": true,
-          "dev": true,
-          "optional": true,
-          "requires": {
-            "wrappy": "1"
-          }
+        "underscore.string": {
+          "version": "2.3.3",
+          "resolved": "https://registry.npmjs.org/underscore.string/-/underscore.string-2.3.3.tgz",
+          "integrity": "sha1-ccCL9rQosRM/N+ePo6Icgvcymw0=",
+          "dev": true
+        }
+      }
+    },
+    "grunt-legacy-log-utils": {
+      "version": "0.1.1",
+      "resolved": "https://registry.npmjs.org/grunt-legacy-log-utils/-/grunt-legacy-log-utils-0.1.1.tgz",
+      "integrity": "sha1-wHBrndkGThFvNvI/5OawSGcsD34=",
+      "dev": true,
+      "requires": {
+        "colors": "0.6.2",
+        "lodash": "2.4.2",
+        "underscore.string": "2.3.3"
+      },
+      "dependencies": {
+        "colors": {
+          "version": "0.6.2",
+          "resolved": "https://registry.npmjs.org/colors/-/colors-0.6.2.tgz",
+          "integrity": "sha1-JCP+ZnisDF2uiFLl0OW+CMmXq8w=",
+          "dev": true
         },
-        "os-homedir": {
-          "version": "1.0.2",
-          "bundled": true,
-          "dev": true,
-          "optional": true
+        "lodash": {
+          "version": "2.4.2",
+          "resolved": "https://registry.npmjs.org/lodash/-/lodash-2.4.2.tgz",
+          "integrity": "sha1-+t2DS5aDBz2hebPq5tnA0VBT9z4=",
+          "dev": true
         },
-        "os-tmpdir": {
-          "version": "1.0.2",
-          "bundled": true,
-          "dev": true,
-          "optional": true
-        },
-        "osenv": {
-          "version": "0.1.5",
-          "bundled": true,
-          "dev": true,
-          "optional": true,
-          "requires": {
-            "os-homedir": "^1.0.0",
-            "os-tmpdir": "^1.0.0"
-          }
-        },
-        "path-is-absolute": {
-          "version": "1.0.1",
-          "bundled": true,
-          "dev": true,
-          "optional": true
+        "underscore.string": {
+          "version": "2.3.3",
+          "resolved": "https://registry.npmjs.org/underscore.string/-/underscore.string-2.3.3.tgz",
+          "integrity": "sha1-ccCL9rQosRM/N+ePo6Icgvcymw0=",
+          "dev": true
+        }
+      }
+    },
+    "grunt-legacy-util": {
+      "version": "0.2.0",
+      "resolved": "https://registry.npmjs.org/grunt-legacy-util/-/grunt-legacy-util-0.2.0.tgz",
+      "integrity": "sha1-kzJIhNv343qf98Am3/RR2UqeVUs=",
+      "dev": true,
+      "requires": {
+        "async": "0.1.22",
+        "exit": "0.1.2",
+        "getobject": "0.1.0",
+        "hooker": "0.2.3",
+        "lodash": "0.9.2",
+        "underscore.string": "2.2.1",
+        "which": "1.0.9"
+      },
+      "dependencies": {
+        "async": {
+          "version": "0.1.22",
+          "resolved": "https://registry.npmjs.org/async/-/async-0.1.22.tgz",
+          "integrity": "sha1-D8GqoIig4+8Ovi2IMbqw3PiEUGE=",
+          "dev": true
         },
-        "process-nextick-args": {
-          "version": "2.0.0",
-          "bundled": true,
-          "dev": true,
-          "optional": true
+        "lodash": {
+          "version": "0.9.2",
+          "resolved": "https://registry.npmjs.org/lodash/-/lodash-0.9.2.tgz",
+          "integrity": "sha1-jzSZxSRdNG1oLlsNO0B2fgnxqSw=",
+          "dev": true
+        }
+      }
+    },
+    "grunt-newer": {
+      "version": "0.7.0",
+      "resolved": "https://registry.npmjs.org/grunt-newer/-/grunt-newer-0.7.0.tgz",
+      "integrity": "sha1-N22dm2TOXGSLa/ob2pj3vCGT5B4=",
+      "dev": true,
+      "requires": {
+        "async": "0.2.10",
+        "rimraf": "2.2.6"
+      },
+      "dependencies": {
+        "rimraf": {
+          "version": "2.2.6",
+          "resolved": "https://registry.npmjs.org/rimraf/-/rimraf-2.2.6.tgz",
+          "integrity": "sha1-xZWXVpsU2VatKcrMQr3d9fDqT0w=",
+          "dev": true
+        }
+      }
+    },
+    "grunt-ng-annotate": {
+      "version": "0.10.0",
+      "resolved": "https://registry.npmjs.org/grunt-ng-annotate/-/grunt-ng-annotate-0.10.0.tgz",
+      "integrity": "sha1-9dw7TDOlZlgkEzELeJhVZuCoS24=",
+      "dev": true,
+      "requires": {
+        "lodash.clonedeep": "3.0.2",
+        "ng-annotate": "0.15.4"
+      }
+    },
+    "grunt-postcss": {
+      "version": "0.7.2",
+      "resolved": "https://registry.npmjs.org/grunt-postcss/-/grunt-postcss-0.7.2.tgz",
+      "integrity": "sha1-V7dke4d9Qq0yz51M0RAID/+0OKs=",
+      "dev": true,
+      "requires": {
+        "chalk": "1.1.3",
+        "diff": "2.2.3",
+        "es6-promise": "3.3.1",
+        "postcss": "5.2.18"
+      }
+    },
+    "grunt-replace": {
+      "version": "1.0.1",
+      "resolved": "https://registry.npmjs.org/grunt-replace/-/grunt-replace-1.0.1.tgz",
+      "integrity": "sha1-kKeVMvuJBB/kJ8h9QlI4sPiGZRo=",
+      "dev": true,
+      "requires": {
+        "applause": "1.2.2",
+        "chalk": "1.1.3",
+        "file-sync-cmp": "0.1.1",
+        "lodash": "4.17.11"
+      }
+    },
+    "grunt-svgmin": {
+      "version": "0.4.0",
+      "resolved": "https://registry.npmjs.org/grunt-svgmin/-/grunt-svgmin-0.4.0.tgz",
+      "integrity": "sha1-8Z0RkwIq4AgOD65dMT4S73yuCq4=",
+      "dev": true,
+      "requires": {
+        "chalk": "0.4.0",
+        "each-async": "0.1.3",
+        "pretty-bytes": "0.1.2",
+        "svgo": "0.4.5"
+      },
+      "dependencies": {
+        "ansi-styles": {
+          "version": "1.0.0",
+          "resolved": "https://registry.npmjs.org/ansi-styles/-/ansi-styles-1.0.0.tgz",
+          "integrity": "sha1-yxAt8cVvUSPquLZ817mAJ6AnkXg=",
+          "dev": true
         },
-        "rc": {
-          "version": "1.2.7",
-          "bundled": true,
+        "argparse": {
+          "version": "0.1.16",
+          "resolved": "https://registry.npmjs.org/argparse/-/argparse-0.1.16.tgz",
+          "integrity": "sha1-z9AeD7uj1srtBJ+9dY1A9lGW9Xw=",
           "dev": true,
-          "optional": true,
           "requires": {
-            "deep-extend": "^0.5.1",
-            "ini": "~1.3.0",
-            "minimist": "^1.2.0",
-            "strip-json-comments": "~2.0.1"
-          },
-          "dependencies": {
-            "minimist": {
-              "version": "1.2.0",
-              "bundled": true,
-              "dev": true,
-              "optional": true
-            }
+            "underscore": "1.7.0",
+            "underscore.string": "2.4.0"
           }
         },
-        "readable-stream": {
-          "version": "2.3.6",
-          "bundled": true,
+        "chalk": {
+          "version": "0.4.0",
+          "resolved": "https://registry.npmjs.org/chalk/-/chalk-0.4.0.tgz",
+          "integrity": "sha1-UZmj3c0MHv4jvAjBsCewYXbgxk8=",
           "dev": true,
-          "optional": true,
           "requires": {
-            "core-util-is": "~1.0.0",
-            "inherits": "~2.0.3",
-            "isarray": "~1.0.0",
-            "process-nextick-args": "~2.0.0",
-            "safe-buffer": "~5.1.1",
-            "string_decoder": "~1.1.1",
-            "util-deprecate": "~1.0.1"
+            "ansi-styles": "1.0.0",
+            "has-color": "0.1.7",
+            "strip-ansi": "0.1.1"
           }
         },
-        "rimraf": {
-          "version": "2.6.2",
-          "bundled": true,
+        "coa": {
+          "version": "0.4.1",
+          "resolved": "https://registry.npmjs.org/coa/-/coa-0.4.1.tgz",
+          "integrity": "sha1-uvb0nHrZ8gxZevObP8HlCQ/og4s=",
           "dev": true,
-          "optional": true,
           "requires": {
-            "glob": "^7.0.5"
+            "q": "0.9.7"
           }
         },
-        "safe-buffer": {
-          "version": "5.1.1",
-          "bundled": true,
+        "colors": {
+          "version": "0.6.2",
+          "resolved": "https://registry.npmjs.org/colors/-/colors-0.6.2.tgz",
+          "integrity": "sha1-JCP+ZnisDF2uiFLl0OW+CMmXq8w=",
           "dev": true
         },
-        "safer-buffer": {
-          "version": "2.1.2",
-          "bundled": true,
-          "dev": true,
-          "optional": true
+        "esprima": {
+          "version": "1.0.4",
+          "resolved": "https://registry.npmjs.org/esprima/-/esprima-1.0.4.tgz",
+          "integrity": "sha1-n1V+CPw7TSbs6d00+Pv0drYlha0=",
+          "dev": true
         },
-        "sax": {
-          "version": "1.2.4",
-          "bundled": true,
+        "js-yaml": {
+          "version": "2.1.3",
+          "resolved": "https://registry.npmjs.org/js-yaml/-/js-yaml-2.1.3.tgz",
+          "integrity": "sha1-D/tWF75VUlh4Bj16Fq7n/dKC6Ew=",
           "dev": true,
-          "optional": true
+          "requires": {
+            "argparse": "0.1.16",
+            "esprima": "1.0.4"
+          }
         },
-        "semver": {
-          "version": "5.5.0",
-          "bundled": true,
-          "dev": true,
-          "optional": true
+        "q": {
+          "version": "0.9.7",
+          "resolved": "https://registry.npmjs.org/q/-/q-0.9.7.tgz",
+          "integrity": "sha1-TeLmyzspCIyeTLwDv51C+5bOL3U=",
+          "dev": true
         },
-        "set-blocking": {
-          "version": "2.0.0",
-          "bundled": true,
-          "dev": true,
-          "optional": true
+        "sax": {
+          "version": "0.6.1",
+          "resolved": "https://registry.npmjs.org/sax/-/sax-0.6.1.tgz",
+          "integrity": "sha1-VjsZx8HeiS4Jv8Ty/DDjwn8JUrk=",
+          "dev": true
         },
-        "signal-exit": {
-          "version": "3.0.2",
-          "bundled": true,
-          "dev": true,
-          "optional": true
+        "strip-ansi": {
+          "version": "0.1.1",
+          "resolved": "https://registry.npmjs.org/strip-ansi/-/strip-ansi-0.1.1.tgz",
+          "integrity": "sha1-OeipjQRNFQZgq+SmgIrPcLt7yZE=",
+          "dev": true
         },
-        "string-width": {
-          "version": "1.0.2",
-          "bundled": true,
+        "svgo": {
+          "version": "0.4.5",
+          "resolved": "https://registry.npmjs.org/svgo/-/svgo-0.4.5.tgz",
+          "integrity": "sha1-ulYVX7FzNyiVbAG0BSIe5+eJoqQ=",
           "dev": true,
-          "optional": true,
           "requires": {
-            "code-point-at": "^1.0.0",
-            "is-fullwidth-code-point": "^1.0.0",
-            "strip-ansi": "^3.0.0"
+            "coa": "0.4.1",
+            "colors": "0.6.2",
+            "js-yaml": "2.1.3",
+            "sax": "0.6.1",
+            "whet.extend": "0.9.9"
           }
         },
-        "string_decoder": {
-          "version": "1.1.1",
-          "bundled": true,
-          "dev": true,
-          "optional": true,
-          "requires": {
-            "safe-buffer": "~5.1.0"
-          }
+        "underscore": {
+          "version": "1.7.0",
+          "resolved": "https://registry.npmjs.org/underscore/-/underscore-1.7.0.tgz",
+          "integrity": "sha1-a7rwh3UA02vjTsqlhODbn+8DUgk=",
+          "dev": true
         },
-        "strip-ansi": {
-          "version": "3.0.1",
-          "bundled": true,
-          "dev": true,
-          "requires": {
-            "ansi-regex": "^2.0.0"
-          }
+        "underscore.string": {
+          "version": "2.4.0",
+          "resolved": "https://registry.npmjs.org/underscore.string/-/underscore.string-2.4.0.tgz",
+          "integrity": "sha1-jN2PusTi0uoefi6Al8QvRCKA+Fs=",
+          "dev": true
+        }
+      }
+    },
+    "grunt-usemin": {
+      "version": "2.6.2",
+      "resolved": "https://registry.npmjs.org/grunt-usemin/-/grunt-usemin-2.6.2.tgz",
+      "integrity": "sha1-KxNroCJkqakdlNQkyNNya9iNt9o=",
+      "dev": true,
+      "requires": {
+        "chalk": "0.5.1",
+        "debug": "2.1.3",
+        "lodash": "2.4.2"
+      },
+      "dependencies": {
+        "ansi-regex": {
+          "version": "0.2.1",
+          "resolved": "https://registry.npmjs.org/ansi-regex/-/ansi-regex-0.2.1.tgz",
+          "integrity": "sha1-DY6UaWej2BQ/k+JOKYUl/BsiNfk=",
+          "dev": true
         },
-        "strip-json-comments": {
-          "version": "2.0.1",
-          "bundled": true,
-          "dev": true,
-          "optional": true
+        "ansi-styles": {
+          "version": "1.1.0",
+          "resolved": "https://registry.npmjs.org/ansi-styles/-/ansi-styles-1.1.0.tgz",
+          "integrity": "sha1-6uy/Zs1waIJ2Cy9GkVgrj1XXp94=",
+          "dev": true
         },
-        "tar": {
-          "version": "4.4.1",
-          "bundled": true,
+        "chalk": {
+          "version": "0.5.1",
+          "resolved": "https://registry.npmjs.org/chalk/-/chalk-0.5.1.tgz",
+          "integrity": "sha1-Zjs6ZItotV0EaQ1JFnqoN4WPIXQ=",
           "dev": true,
-          "optional": true,
           "requires": {
-            "chownr": "^1.0.1",
-            "fs-minipass": "^1.2.5",
-            "minipass": "^2.2.4",
-            "minizlib": "^1.1.0",
-            "mkdirp": "^0.5.0",
-            "safe-buffer": "^5.1.1",
-            "yallist": "^3.0.2"
+            "ansi-styles": "1.1.0",
+            "escape-string-regexp": "1.0.5",
+            "has-ansi": "0.1.0",
+            "strip-ansi": "0.3.0",
+            "supports-color": "0.2.0"
           }
         },
-        "util-deprecate": {
-          "version": "1.0.2",
-          "bundled": true,
+        "debug": {
+          "version": "2.1.3",
+          "resolved": "https://registry.npmjs.org/debug/-/debug-2.1.3.tgz",
+          "integrity": "sha1-zoqxte6PvuK/o7Yzyrk9NmtjQY4=",
           "dev": true,
-          "optional": true
+          "requires": {
+            "ms": "0.7.0"
+          }
         },
-        "wide-align": {
-          "version": "1.1.2",
-          "bundled": true,
+        "has-ansi": {
+          "version": "0.1.0",
+          "resolved": "https://registry.npmjs.org/has-ansi/-/has-ansi-0.1.0.tgz",
+          "integrity": "sha1-hPJlqujA5qiKEtcCKJS3VoiUxi4=",
           "dev": true,
-          "optional": true,
           "requires": {
-            "string-width": "^1.0.2"
+            "ansi-regex": "0.2.1"
           }
         },
-        "wrappy": {
-          "version": "1.0.2",
-          "bundled": true,
+        "lodash": {
+          "version": "2.4.2",
+          "resolved": "https://registry.npmjs.org/lodash/-/lodash-2.4.2.tgz",
+          "integrity": "sha1-+t2DS5aDBz2hebPq5tnA0VBT9z4=",
           "dev": true
         },
-        "yallist": {
-          "version": "3.0.2",
-          "bundled": true,
+        "ms": {
+          "version": "0.7.0",
+          "resolved": "https://registry.npmjs.org/ms/-/ms-0.7.0.tgz",
+          "integrity": "sha1-hlvpTC5zl62KV9pqYzpuLzB5i4M=",
+          "dev": true
+        },
+        "strip-ansi": {
+          "version": "0.3.0",
+          "resolved": "https://registry.npmjs.org/strip-ansi/-/strip-ansi-0.3.0.tgz",
+          "integrity": "sha1-JfSOoiynkYfzF0pNuHWTR7sSYiA=",
+          "dev": true,
+          "requires": {
+            "ansi-regex": "0.2.1"
+          }
+        },
+        "supports-color": {
+          "version": "0.2.0",
+          "resolved": "https://registry.npmjs.org/supports-color/-/supports-color-0.2.0.tgz",
+          "integrity": "sha1-2S3iaU6z9nMjlz1649i1W0wiGQo=",
           "dev": true
         }
       }
     },
-    "function-bind": {
-      "version": "1.1.1",
-      "resolved": "https://registry.npmjs.org/function-bind/-/function-bind-1.1.1.tgz",
-      "integrity": "sha512-yIovAzMX49sF8Yl58fSCWJ5svSLuaibPxXQJFLmBObTuCr0Mf1KiPopGM9NiFjiYBCbfaa2Fh6breQ6ANVTI0A==",
-      "dev": true
-    },
-    "gaze": {
-      "version": "0.5.2",
-      "resolved": "https://registry.npmjs.org/gaze/-/gaze-0.5.2.tgz",
-      "integrity": "sha1-QLcJU30k0dRXZ9takIaJ3+aaxE8=",
+    "grunt-wiredep": {
+      "version": "2.0.0",
+      "resolved": "https://registry.npmjs.org/grunt-wiredep/-/grunt-wiredep-2.0.0.tgz",
+      "integrity": "sha1-ID9vYT95nW3XLOBE0NzvZNrx8uU=",
       "dev": true,
       "requires": {
-        "globule": "~0.1.0"
+        "wiredep": "2.2.2"
       }
     },
-    "generate-function": {
-      "version": "2.3.1",
-      "resolved": "https://registry.npmjs.org/generate-function/-/generate-function-2.3.1.tgz",
-      "integrity": "sha512-eeB5GfMNeevm/GRYq20ShmsaGcmI81kIX2K9XQx5miC8KdHaC6Jm0qQ8ZNeGOi7wYB8OsdxKs+Y2oVuTFuVwKQ==",
+    "gzip-size": {
+      "version": "0.1.1",
+      "resolved": "https://registry.npmjs.org/gzip-size/-/gzip-size-0.1.1.tgz",
+      "integrity": "sha1-rjNIO2/IIk6DQilt4Qjvk3V/duA=",
       "dev": true,
       "requires": {
-        "is-property": "^1.0.2"
+        "concat-stream": "1.6.2",
+        "zlib-browserify": "0.0.3"
       }
     },
-    "generate-object-property": {
-      "version": "1.2.0",
-      "resolved": "https://registry.npmjs.org/generate-object-property/-/generate-object-property-1.2.0.tgz",
-      "integrity": "sha1-nA4cQDCM6AT0eDYYuTf6iPmdUNA=",
+    "handle-thing": {
+      "version": "1.2.5",
+      "resolved": "https://registry.npmjs.org/handle-thing/-/handle-thing-1.2.5.tgz",
+      "integrity": "sha1-/Xqtcmvxpf0W38KbL3pmAdJxOcQ=",
+      "dev": true
+    },
+    "handlebars": {
+      "version": "4.0.12",
+      "resolved": "https://registry.npmjs.org/handlebars/-/handlebars-4.0.12.tgz",
+      "integrity": "sha512-RhmTekP+FZL+XNhwS1Wf+bTTZpdLougwt5pcgA1tuz6Jcx0fpH/7z0qd71RKnZHBCxIRBHfBOnio4gViPemNzA==",
       "dev": true,
       "requires": {
-        "is-property": "^1.0.0"
+        "async": "2.6.1",
+        "optimist": "0.6.1",
+        "source-map": "0.6.1",
+        "uglify-js": "3.4.9"
+      },
+      "dependencies": {
+        "async": {
+          "version": "2.6.1",
+          "resolved": "https://registry.npmjs.org/async/-/async-2.6.1.tgz",
+          "integrity": "sha512-fNEiL2+AZt6AlAw/29Cr0UDe4sRAHCpEHh54WMz+Bb7QfNcFw4h3loofyJpLeQs4Yx7yuqu/2dLgM5hKOs6HlQ==",
+          "dev": true,
+          "requires": {
+            "lodash": "4.17.11"
+          }
+        },
+        "commander": {
+          "version": "2.17.1",
+          "resolved": "https://registry.npmjs.org/commander/-/commander-2.17.1.tgz",
+          "integrity": "sha512-wPMUt6FnH2yzG95SA6mzjQOEKUU3aLaDEmzs1ti+1E9h+CsrZghRlqEM/EJ4KscsQVG8uNN4uVreUeT8+drlgg==",
+          "dev": true,
+          "optional": true
+        },
+        "source-map": {
+          "version": "0.6.1",
+          "resolved": "https://registry.npmjs.org/source-map/-/source-map-0.6.1.tgz",
+          "integrity": "sha512-UjgapumWlbMhkBgzT7Ykc5YXUT46F0iKu8SGXq0bcwP5dz/h0Plj6enJqjz1Zbq2l5WaqYnrVbwWOWMyF3F47g==",
+          "dev": true
+        },
+        "uglify-js": {
+          "version": "3.4.9",
+          "resolved": "https://registry.npmjs.org/uglify-js/-/uglify-js-3.4.9.tgz",
+          "integrity": "sha512-8CJsbKOtEbnJsTyv6LE6m6ZKniqMiFWmm9sRbopbkGs3gMPPfd3Fh8iIA4Ykv5MgaTbqHr4BaoGLJLZNhsrW1Q==",
+          "dev": true,
+          "optional": true,
+          "requires": {
+            "commander": "2.17.1",
+            "source-map": "0.6.1"
+          }
+        }
       }
     },
-    "get-caller-file": {
-      "version": "1.0.3",
-      "resolved": "https://registry.npmjs.org/get-caller-file/-/get-caller-file-1.0.3.tgz",
-      "integrity": "sha512-3t6rVToeoZfYSGd8YoLFR2DJkiQrIiUrGcjvFX2mDw3bn6k2OtwHN0TNCLbBO+w8qTvimhDkv+LSscbJY1vE6w==",
-      "dev": true
-    },
-    "get-stdin": {
-      "version": "4.0.1",
-      "resolved": "https://registry.npmjs.org/get-stdin/-/get-stdin-4.0.1.tgz",
-      "integrity": "sha1-uWjGsKBDhDJJAui/Gl3zJXmkUP4=",
-      "dev": true
-    },
-    "get-stream": {
-      "version": "3.0.0",
-      "resolved": "https://registry.npmjs.org/get-stream/-/get-stream-3.0.0.tgz",
-      "integrity": "sha1-jpQ9E1jcN1VQVOy+LtsFqhdO3hQ=",
-      "dev": true
-    },
-    "get-value": {
-      "version": "2.0.6",
-      "resolved": "https://registry.npmjs.org/get-value/-/get-value-2.0.6.tgz",
-      "integrity": "sha1-3BXKHGcjh8p2vTesCjlbogQqLCg=",
-      "dev": true
-    },
-    "getobject": {
-      "version": "0.1.0",
-      "resolved": "https://registry.npmjs.org/getobject/-/getobject-0.1.0.tgz",
-      "integrity": "sha1-BHpEl4n6Fg0Bj1SG7ZEyC27HiFw=",
+    "har-schema": {
+      "version": "2.0.0",
+      "resolved": "https://registry.npmjs.org/har-schema/-/har-schema-2.0.0.tgz",
+      "integrity": "sha1-qUwiJOvKwEeCoNkDVSHyRzW37JI=",
       "dev": true
     },
-    "getpass": {
-      "version": "0.1.7",
-      "resolved": "https://registry.npmjs.org/getpass/-/getpass-0.1.7.tgz",
-      "integrity": "sha1-Xv+OPmhNVprkyysSgmBOi6YhSfo=",
+    "har-validator": {
+      "version": "5.1.0",
+      "resolved": "https://registry.npmjs.org/har-validator/-/har-validator-5.1.0.tgz",
+      "integrity": "sha512-+qnmNjI4OfH2ipQ9VQOw23bBd/ibtfbVdK2fYbY4acTDqKTW/YDp9McimZdDbG8iV9fZizUqQMD5xvriB146TA==",
       "dev": true,
       "requires": {
-        "assert-plus": "^1.0.0"
+        "ajv": "5.5.2",
+        "har-schema": "2.0.0"
+      },
+      "dependencies": {
+        "ajv": {
+          "version": "5.5.2",
+          "resolved": "https://registry.npmjs.org/ajv/-/ajv-5.5.2.tgz",
+          "integrity": "sha1-c7Xuyj+rZT49P5Qis0GtQiBdyWU=",
+          "dev": true,
+          "requires": {
+            "co": "4.6.0",
+            "fast-deep-equal": "1.1.0",
+            "fast-json-stable-stringify": "2.0.0",
+            "json-schema-traverse": "0.3.1"
+          }
+        },
+        "fast-deep-equal": {
+          "version": "1.1.0",
+          "resolved": "https://registry.npmjs.org/fast-deep-equal/-/fast-deep-equal-1.1.0.tgz",
+          "integrity": "sha1-wFNHeBfIa1HaqFPIHgWbcz0CNhQ=",
+          "dev": true
+        },
+        "json-schema-traverse": {
+          "version": "0.3.1",
+          "resolved": "https://registry.npmjs.org/json-schema-traverse/-/json-schema-traverse-0.3.1.tgz",
+          "integrity": "sha1-NJptRMU6Ud6JtAgFxdXlm0F9M0A=",
+          "dev": true
+        }
       }
     },
-    "github-markdown-css": {
-      "version": "2.6.0",
-      "resolved": "https://registry.npmjs.org/github-markdown-css/-/github-markdown-css-2.6.0.tgz",
-      "integrity": "sha1-zcdLq1ZrA51/u3RgH3ghsQnPWRg="
-    },
-    "glob": {
-      "version": "3.2.11",
-      "resolved": "https://registry.npmjs.org/glob/-/glob-3.2.11.tgz",
-      "integrity": "sha1-Spc/Y1uRkPcV0QmH1cAP0oFevj0=",
+    "has": {
+      "version": "1.0.3",
+      "resolved": "https://registry.npmjs.org/has/-/has-1.0.3.tgz",
+      "integrity": "sha512-f2dvO0VU6Oej7RkWJGrehjbzMAjFp5/VKPp5tTpWIV4JHHZK1/BxbFRtf/siA2SWTe09caDmVtYYzWEIbBS4zw==",
+      "dev": true,
       "requires": {
-        "inherits": "2",
-        "minimatch": "0.3"
+        "function-bind": "1.1.1"
       }
     },
-    "glob-base": {
-      "version": "0.3.0",
-      "resolved": "https://registry.npmjs.org/glob-base/-/glob-base-0.3.0.tgz",
-      "integrity": "sha1-27Fk9iIbHAscz4Kuoyi0l98Oo8Q=",
+    "has-ansi": {
+      "version": "2.0.0",
+      "resolved": "https://registry.npmjs.org/has-ansi/-/has-ansi-2.0.0.tgz",
+      "integrity": "sha1-NPUEnOHs3ysGSa8+8k5F7TVBbZE=",
       "dev": true,
       "requires": {
-        "glob-parent": "^2.0.0",
-        "is-glob": "^2.0.0"
+        "ansi-regex": "2.1.1"
       }
     },
-    "glob-parent": {
-      "version": "2.0.0",
-      "resolved": "https://registry.npmjs.org/glob-parent/-/glob-parent-2.0.0.tgz",
-      "integrity": "sha1-gTg9ctsFT8zPUzbaqQLxgvbtuyg=",
+    "has-binary2": {
+      "version": "1.0.3",
+      "resolved": "https://registry.npmjs.org/has-binary2/-/has-binary2-1.0.3.tgz",
+      "integrity": "sha512-G1LWKhDSvhGeAQ8mPVQlqNcOB2sJdwATtZKl2pDKKHfpf/rYj24lkinxf69blJbnsvtqqNU+L3SL50vzZhXOnw==",
       "dev": true,
       "requires": {
-        "is-glob": "^2.0.0"
+        "isarray": "2.0.1"
+      },
+      "dependencies": {
+        "isarray": {
+          "version": "2.0.1",
+          "resolved": "https://registry.npmjs.org/isarray/-/isarray-2.0.1.tgz",
+          "integrity": "sha1-o32U7ZzaLVmGXJ92/llu4fM4dB4=",
+          "dev": true
+        }
       }
     },
-    "global-modules-path": {
-      "version": "2.3.0",
-      "resolved": "https://registry.npmjs.org/global-modules-path/-/global-modules-path-2.3.0.tgz",
-      "integrity": "sha512-HchvMJNYh9dGSCy8pOQ2O8u/hoXaL+0XhnrwH0RyLiSXMMTl9W3N6KUU73+JFOg5PGjtzl6VZzUQsnrpm7Szag==",
+    "has-color": {
+      "version": "0.1.7",
+      "resolved": "https://registry.npmjs.org/has-color/-/has-color-0.1.7.tgz",
+      "integrity": "sha1-ZxRKUmDDT8PMpnfQQdr1L+e3iy8=",
       "dev": true
     },
-    "globals": {
-      "version": "9.18.0",
-      "resolved": "https://registry.npmjs.org/globals/-/globals-9.18.0.tgz",
-      "integrity": "sha512-S0nG3CLEQiY/ILxqtztTWH/3iRRdyBLw6KMDxnKMchrtbj2OFmehVh0WUCfW3DUrIgx/qFrJPICrq4Z4sTR9UQ==",
+    "has-cors": {
+      "version": "1.1.0",
+      "resolved": "https://registry.npmjs.org/has-cors/-/has-cors-1.1.0.tgz",
+      "integrity": "sha1-XkdHk/fqmEPRu5nCPu9J/xJv/zk=",
       "dev": true
     },
-    "globby": {
-      "version": "7.1.1",
-      "resolved": "https://registry.npmjs.org/globby/-/globby-7.1.1.tgz",
-      "integrity": "sha1-+yzP+UAfhgCUXfral0QMypcrhoA=",
+    "has-flag": {
+      "version": "1.0.0",
+      "resolved": "https://registry.npmjs.org/has-flag/-/has-flag-1.0.0.tgz",
+      "integrity": "sha1-nZ55MWXOAXoA8AQYxD+UKnsdEfo=",
+      "dev": true
+    },
+    "has-symbols": {
+      "version": "1.0.0",
+      "resolved": "https://registry.npmjs.org/has-symbols/-/has-symbols-1.0.0.tgz",
+      "integrity": "sha1-uhqPGvKg/DllD1yFA2dwQSIGO0Q=",
+      "dev": true
+    },
+    "has-value": {
+      "version": "1.0.0",
+      "resolved": "https://registry.npmjs.org/has-value/-/has-value-1.0.0.tgz",
+      "integrity": "sha1-GLKB2lhbHFxR3vJMkw7SmgvmsXc=",
       "dev": true,
       "requires": {
-        "array-union": "^1.0.1",
-        "dir-glob": "^2.0.0",
-        "glob": "^7.1.2",
-        "ignore": "^3.3.5",
-        "pify": "^3.0.0",
-        "slash": "^1.0.0"
+        "get-value": "2.0.6",
+        "has-values": "1.0.0",
+        "isobject": "3.0.1"
       },
       "dependencies": {
-        "glob": {
-          "version": "7.1.3",
-          "resolved": "https://registry.npmjs.org/glob/-/glob-7.1.3.tgz",
-          "integrity": "sha512-vcfuiIxogLV4DlGBHIUOwI0IbrJ8HWPc4MU7HzviGeNho/UJDfi6B5p3sHeWIQ0KGIU0Jpxi5ZHxemQfLkkAwQ==",
+        "isobject": {
+          "version": "3.0.1",
+          "resolved": "https://registry.npmjs.org/isobject/-/isobject-3.0.1.tgz",
+          "integrity": "sha1-TkMekrEalzFjaqH5yNHMvP2reN8=",
+          "dev": true
+        }
+      }
+    },
+    "has-values": {
+      "version": "1.0.0",
+      "resolved": "https://registry.npmjs.org/has-values/-/has-values-1.0.0.tgz",
+      "integrity": "sha1-lbC2P+whRmGab+V/51Yo1aOe/k8=",
+      "dev": true,
+      "requires": {
+        "is-number": "3.0.0",
+        "kind-of": "4.0.0"
+      },
+      "dependencies": {
+        "is-number": {
+          "version": "3.0.0",
+          "resolved": "https://registry.npmjs.org/is-number/-/is-number-3.0.0.tgz",
+          "integrity": "sha1-JP1iAaR4LPUFYcgQJ2r8fRLXEZU=",
           "dev": true,
           "requires": {
-            "fs.realpath": "^1.0.0",
-            "inflight": "^1.0.4",
-            "inherits": "2",
-            "minimatch": "^3.0.4",
-            "once": "^1.3.0",
-            "path-is-absolute": "^1.0.0"
+            "kind-of": "3.2.2"
+          },
+          "dependencies": {
+            "kind-of": {
+              "version": "3.2.2",
+              "resolved": "https://registry.npmjs.org/kind-of/-/kind-of-3.2.2.tgz",
+              "integrity": "sha1-MeohpzS6ubuw8yRm2JOupR5KPGQ=",
+              "dev": true,
+              "requires": {
+                "is-buffer": "1.1.6"
+              }
+            }
           }
         },
-        "minimatch": {
-          "version": "3.0.4",
-          "resolved": "https://registry.npmjs.org/minimatch/-/minimatch-3.0.4.tgz",
-          "integrity": "sha512-yJHVQEhyqPLUTgt9B83PXu6W3rx4MvvHvSUvToogpwoGDOUQ+yDrR0HRot+yOCdCO7u4hX3pWft6kWBBcqh0UA==",
+        "kind-of": {
+          "version": "4.0.0",
+          "resolved": "https://registry.npmjs.org/kind-of/-/kind-of-4.0.0.tgz",
+          "integrity": "sha1-IIE989cSkosgc3hpGkUGb65y3Vc=",
           "dev": true,
           "requires": {
-            "brace-expansion": "^1.1.7"
+            "is-buffer": "1.1.6"
           }
         }
       }
     },
-    "globule": {
-      "version": "0.1.0",
-      "resolved": "https://registry.npmjs.org/globule/-/globule-0.1.0.tgz",
-      "integrity": "sha1-2cjt3h2nnRJaFRt5UzuXhnY0auU=",
+    "hash-base": {
+      "version": "3.0.4",
+      "resolved": "https://registry.npmjs.org/hash-base/-/hash-base-3.0.4.tgz",
+      "integrity": "sha1-X8hoaEfs1zSZQDMZprCj8/auSRg=",
+      "dev": true,
+      "requires": {
+        "inherits": "2.0.3",
+        "safe-buffer": "5.1.2"
+      }
+    },
+    "hash.js": {
+      "version": "1.1.5",
+      "resolved": "https://registry.npmjs.org/hash.js/-/hash.js-1.1.5.tgz",
+      "integrity": "sha512-eWI5HG9Np+eHV1KQhisXWwM+4EPPYe5dFX1UZZH7k/E3JzDEazVH+VGlZi6R94ZqImq+A3D1mCEtrFIfg/E7sA==",
+      "dev": true,
+      "requires": {
+        "inherits": "2.0.3",
+        "minimalistic-assert": "1.0.1"
+      }
+    },
+    "hasha": {
+      "version": "2.2.0",
+      "resolved": "https://registry.npmjs.org/hasha/-/hasha-2.2.0.tgz",
+      "integrity": "sha1-eNfL/B5tZjA/55g3NlmEUXsvbuE=",
+      "dev": true,
+      "requires": {
+        "is-stream": "1.1.0",
+        "pinkie-promise": "2.0.1"
+      }
+    },
+    "he": {
+      "version": "1.2.0",
+      "resolved": "https://registry.npmjs.org/he/-/he-1.2.0.tgz",
+      "integrity": "sha512-F/1DnUGPopORZi0ni+CvrCgHQ5FyEAHRLSApuYWMmrbSwoN2Mn/7k+Gl38gJnR7yyDZk6WLXwiGod1JOWNDKGw==",
+      "dev": true
+    },
+    "headroom.js": {
+      "version": "0.9.4",
+      "resolved": "https://registry.npmjs.org/headroom.js/-/headroom.js-0.9.4.tgz",
+      "integrity": "sha1-DE5rRWO7ad9Vrs3vq6MidWby31o="
+    },
+    "hmac-drbg": {
+      "version": "1.0.1",
+      "resolved": "https://registry.npmjs.org/hmac-drbg/-/hmac-drbg-1.0.1.tgz",
+      "integrity": "sha1-0nRXAQJabHdabFRXk+1QL8DGSaE=",
+      "dev": true,
+      "requires": {
+        "hash.js": "1.1.5",
+        "minimalistic-assert": "1.0.1",
+        "minimalistic-crypto-utils": "1.0.1"
+      }
+    },
+    "home-or-tmp": {
+      "version": "2.0.0",
+      "resolved": "https://registry.npmjs.org/home-or-tmp/-/home-or-tmp-2.0.0.tgz",
+      "integrity": "sha1-42w/LSyufXRqhX440Y1fMqeILbg=",
+      "dev": true,
+      "requires": {
+        "os-homedir": "1.0.2",
+        "os-tmpdir": "1.0.2"
+      }
+    },
+    "hooker": {
+      "version": "0.2.3",
+      "resolved": "https://registry.npmjs.org/hooker/-/hooker-0.2.3.tgz",
+      "integrity": "sha1-uDT3I8xKJCqmWWNFnfbZhMXT2Vk=",
+      "dev": true
+    },
+    "hosted-git-info": {
+      "version": "2.7.1",
+      "resolved": "https://registry.npmjs.org/hosted-git-info/-/hosted-git-info-2.7.1.tgz",
+      "integrity": "sha512-7T/BxH19zbcCTa8XkMlbK5lTo1WtgkFi3GvdWEyNuc4Vex7/9Dqbnpsf4JMydcfj9HCg4zUWFTL3Za6lapg5/w==",
+      "dev": true
+    },
+    "hpack.js": {
+      "version": "2.1.6",
+      "resolved": "https://registry.npmjs.org/hpack.js/-/hpack.js-2.1.6.tgz",
+      "integrity": "sha1-h3dMCUnlE/QuhFdbPEVoH63ioLI=",
       "dev": true,
       "requires": {
-        "glob": "~3.1.21",
-        "lodash": "~1.0.1",
-        "minimatch": "~0.2.11"
+        "inherits": "2.0.3",
+        "obuf": "1.1.2",
+        "readable-stream": "2.3.6",
+        "wbuf": "1.7.3"
       },
       "dependencies": {
-        "glob": {
-          "version": "3.1.21",
-          "resolved": "https://registry.npmjs.org/glob/-/glob-3.1.21.tgz",
-          "integrity": "sha1-0p4KBV3qUTj00H7UDomC6DwgZs0=",
+        "isarray": {
+          "version": "1.0.0",
+          "resolved": "https://registry.npmjs.org/isarray/-/isarray-1.0.0.tgz",
+          "integrity": "sha1-u5NdSFgsuhaMBoNJV6VKPgcSTxE=",
+          "dev": true
+        },
+        "readable-stream": {
+          "version": "2.3.6",
+          "resolved": "https://registry.npmjs.org/readable-stream/-/readable-stream-2.3.6.tgz",
+          "integrity": "sha512-tQtKA9WIAhBF3+VLAseyMqZeBjW0AHJoxOtYqSUZNJxauErmLbVm2FW1y+J/YA9dUrAC39ITejlZWhVIwawkKw==",
           "dev": true,
           "requires": {
-            "graceful-fs": "~1.2.0",
-            "inherits": "1",
-            "minimatch": "~0.2.11"
+            "core-util-is": "1.0.2",
+            "inherits": "2.0.3",
+            "isarray": "1.0.0",
+            "process-nextick-args": "2.0.0",
+            "safe-buffer": "5.1.2",
+            "string_decoder": "1.1.1",
+            "util-deprecate": "1.0.2"
           }
         },
-        "graceful-fs": {
-          "version": "1.2.3",
-          "resolved": "https://registry.npmjs.org/graceful-fs/-/graceful-fs-1.2.3.tgz",
-          "integrity": "sha1-FaSAaldUfLLS2/J/QuiajDRRs2Q=",
-          "dev": true
-        },
-        "inherits": {
-          "version": "1.0.2",
-          "resolved": "https://registry.npmjs.org/inherits/-/inherits-1.0.2.tgz",
-          "integrity": "sha1-ykMJ2t7mtUzAuNJH6NfHoJdb3Js=",
-          "dev": true
-        },
-        "lodash": {
-          "version": "1.0.2",
-          "resolved": "https://registry.npmjs.org/lodash/-/lodash-1.0.2.tgz",
-          "integrity": "sha1-j1dWDIO1n8JwvT1WG2kAQ0MOJVE=",
-          "dev": true
-        },
-        "minimatch": {
-          "version": "0.2.14",
-          "resolved": "https://registry.npmjs.org/minimatch/-/minimatch-0.2.14.tgz",
-          "integrity": "sha1-x054BXT2PG+aCQ6Q775u9TpqdWo=",
+        "string_decoder": {
+          "version": "1.1.1",
+          "resolved": "https://registry.npmjs.org/string_decoder/-/string_decoder-1.1.1.tgz",
+          "integrity": "sha512-n/ShnvDi6FHbbVfviro+WojiFzv+s8MPMHBczVePfUpDJLwoLT0ht1l4YwBCbi8pJAveEEdnkHyPyTP/mzRfwg==",
           "dev": true,
           "requires": {
-            "lru-cache": "2",
-            "sigmund": "~1.0.0"
+            "safe-buffer": "5.1.2"
           }
         }
       }
     },
-    "graceful-fs": {
-      "version": "4.1.15",
-      "resolved": "https://registry.npmjs.org/graceful-fs/-/graceful-fs-4.1.15.tgz",
-      "integrity": "sha512-6uHUhOPEBgQ24HM+r6b/QwWfZq+yiFcipKFrOFiBEnWdy5sdzYoi+pJeQaPI5qOLRFqWmAXUPQNsielzdLoecA==",
+    "html-comment-regex": {
+      "version": "1.1.2",
+      "resolved": "https://registry.npmjs.org/html-comment-regex/-/html-comment-regex-1.1.2.tgz",
+      "integrity": "sha512-P+M65QY2JQ5Y0G9KKdlDpo0zK+/OHptU5AaBwUfAIDJZk1MYf32Frm84EcOytfJE0t5JvkAnKlmjsXDnWzCJmQ==",
       "dev": true
     },
-    "grunt": {
-      "version": "0.4.5",
-      "resolved": "https://registry.npmjs.org/grunt/-/grunt-0.4.5.tgz",
-      "integrity": "sha1-VpN81RlDJK3/bSB2MYMqnWuk5/A=",
+    "html-entities": {
+      "version": "1.2.1",
+      "resolved": "https://registry.npmjs.org/html-entities/-/html-entities-1.2.1.tgz",
+      "integrity": "sha1-DfKTUfByEWNRXfueVUPl9u7VFi8=",
+      "dev": true
+    },
+    "html-minifier": {
+      "version": "0.6.9",
+      "resolved": "https://registry.npmjs.org/html-minifier/-/html-minifier-0.6.9.tgz",
+      "integrity": "sha1-UQXcI29efhqLplHUq5gThvx6vlM=",
+      "requires": {
+        "change-case": "2.1.6",
+        "clean-css": "2.2.23",
+        "cli": "0.6.6",
+        "relateurl": "0.2.7",
+        "uglify-js": "2.4.24"
+      }
+    },
+    "html-webpack-plugin": {
+      "version": "3.2.0",
+      "resolved": "https://registry.npmjs.org/html-webpack-plugin/-/html-webpack-plugin-3.2.0.tgz",
+      "integrity": "sha1-sBq71yOsqqeze2r0SS69oD2d03s=",
       "dev": true,
       "requires": {
-        "async": "~0.1.22",
-        "coffee-script": "~1.3.3",
-        "colors": "~0.6.2",
-        "dateformat": "1.0.2-1.2.3",
-        "eventemitter2": "~0.4.13",
-        "exit": "~0.1.1",
-        "findup-sync": "~0.1.2",
-        "getobject": "~0.1.0",
-        "glob": "~3.1.21",
-        "grunt-legacy-log": "~0.1.0",
-        "grunt-legacy-util": "~0.2.0",
-        "hooker": "~0.2.3",
-        "iconv-lite": "~0.2.11",
-        "js-yaml": "~2.0.5",
-        "lodash": "~0.9.2",
-        "minimatch": "~0.2.12",
-        "nopt": "~1.0.10",
-        "rimraf": "~2.2.8",
-        "underscore.string": "~2.2.1",
-        "which": "~1.0.5"
+        "html-minifier": "3.5.21",
+        "loader-utils": "0.2.17",
+        "lodash": "4.17.11",
+        "pretty-error": "2.1.1",
+        "tapable": "1.1.0",
+        "toposort": "1.0.7",
+        "util.promisify": "1.0.0"
       },
       "dependencies": {
-        "argparse": {
-          "version": "0.1.16",
-          "resolved": "https://registry.npmjs.org/argparse/-/argparse-0.1.16.tgz",
-          "integrity": "sha1-z9AeD7uj1srtBJ+9dY1A9lGW9Xw=",
+        "camel-case": {
+          "version": "3.0.0",
+          "resolved": "https://registry.npmjs.org/camel-case/-/camel-case-3.0.0.tgz",
+          "integrity": "sha1-yjw2iKTpzzpM2nd9xNy8cTJJz3M=",
           "dev": true,
           "requires": {
-            "underscore": "~1.7.0",
-            "underscore.string": "~2.4.0"
-          },
-          "dependencies": {
-            "underscore.string": {
-              "version": "2.4.0",
-              "resolved": "https://registry.npmjs.org/underscore.string/-/underscore.string-2.4.0.tgz",
-              "integrity": "sha1-jN2PusTi0uoefi6Al8QvRCKA+Fs=",
-              "dev": true
-            }
+            "no-case": "2.3.2",
+            "upper-case": "1.1.3"
           }
         },
-        "async": {
-          "version": "0.1.22",
-          "resolved": "https://registry.npmjs.org/async/-/async-0.1.22.tgz",
-          "integrity": "sha1-D8GqoIig4+8Ovi2IMbqw3PiEUGE=",
-          "dev": true
-        },
-        "colors": {
-          "version": "0.6.2",
-          "resolved": "https://registry.npmjs.org/colors/-/colors-0.6.2.tgz",
-          "integrity": "sha1-JCP+ZnisDF2uiFLl0OW+CMmXq8w=",
-          "dev": true
-        },
-        "esprima": {
-          "version": "1.0.4",
-          "resolved": "https://registry.npmjs.org/esprima/-/esprima-1.0.4.tgz",
-          "integrity": "sha1-n1V+CPw7TSbs6d00+Pv0drYlha0=",
-          "dev": true
-        },
-        "glob": {
-          "version": "3.1.21",
-          "resolved": "https://registry.npmjs.org/glob/-/glob-3.1.21.tgz",
-          "integrity": "sha1-0p4KBV3qUTj00H7UDomC6DwgZs0=",
+        "clean-css": {
+          "version": "4.2.1",
+          "resolved": "https://registry.npmjs.org/clean-css/-/clean-css-4.2.1.tgz",
+          "integrity": "sha512-4ZxI6dy4lrY6FHzfiy1aEOXgu4LIsW2MhwG0VBKdcoGoH/XLFgaHSdLTGr4O8Be6A8r3MOphEiI8Gc1n0ecf3g==",
           "dev": true,
           "requires": {
-            "graceful-fs": "~1.2.0",
-            "inherits": "1",
-            "minimatch": "~0.2.11"
+            "source-map": "0.6.1"
           }
         },
-        "graceful-fs": {
-          "version": "1.2.3",
-          "resolved": "https://registry.npmjs.org/graceful-fs/-/graceful-fs-1.2.3.tgz",
-          "integrity": "sha1-FaSAaldUfLLS2/J/QuiajDRRs2Q=",
-          "dev": true
-        },
-        "iconv-lite": {
-          "version": "0.2.11",
-          "resolved": "https://registry.npmjs.org/iconv-lite/-/iconv-lite-0.2.11.tgz",
-          "integrity": "sha1-HOYKOleGSiktEyH/RgnKS7llrcg=",
-          "dev": true
-        },
-        "inherits": {
-          "version": "1.0.2",
-          "resolved": "https://registry.npmjs.org/inherits/-/inherits-1.0.2.tgz",
-          "integrity": "sha1-ykMJ2t7mtUzAuNJH6NfHoJdb3Js=",
+        "commander": {
+          "version": "2.17.1",
+          "resolved": "https://registry.npmjs.org/commander/-/commander-2.17.1.tgz",
+          "integrity": "sha512-wPMUt6FnH2yzG95SA6mzjQOEKUU3aLaDEmzs1ti+1E9h+CsrZghRlqEM/EJ4KscsQVG8uNN4uVreUeT8+drlgg==",
           "dev": true
         },
-        "js-yaml": {
-          "version": "2.0.5",
-          "resolved": "https://registry.npmjs.org/js-yaml/-/js-yaml-2.0.5.tgz",
-          "integrity": "sha1-olrmUJmZ6X3yeMZxnaEb0Gh3Q6g=",
+        "html-minifier": {
+          "version": "3.5.21",
+          "resolved": "https://registry.npmjs.org/html-minifier/-/html-minifier-3.5.21.tgz",
+          "integrity": "sha512-LKUKwuJDhxNa3uf/LPR/KVjm/l3rBqtYeCOAekvG8F1vItxMUpueGd94i/asDDr8/1u7InxzFA5EeGjhhG5mMA==",
           "dev": true,
           "requires": {
-            "argparse": "~ 0.1.11",
-            "esprima": "~ 1.0.2"
+            "camel-case": "3.0.0",
+            "clean-css": "4.2.1",
+            "commander": "2.17.1",
+            "he": "1.2.0",
+            "param-case": "2.1.1",
+            "relateurl": "0.2.7",
+            "uglify-js": "3.4.9"
           }
         },
-        "lodash": {
-          "version": "0.9.2",
-          "resolved": "https://registry.npmjs.org/lodash/-/lodash-0.9.2.tgz",
-          "integrity": "sha1-jzSZxSRdNG1oLlsNO0B2fgnxqSw=",
-          "dev": true
+        "loader-utils": {
+          "version": "0.2.17",
+          "resolved": "https://registry.npmjs.org/loader-utils/-/loader-utils-0.2.17.tgz",
+          "integrity": "sha1-+G5jdNQyBabmxg6RlvF8Apm/s0g=",
+          "dev": true,
+          "requires": {
+            "big.js": "3.2.0",
+            "emojis-list": "2.1.0",
+            "json5": "0.5.1",
+            "object-assign": "4.1.1"
+          }
         },
-        "minimatch": {
-          "version": "0.2.14",
-          "resolved": "https://registry.npmjs.org/minimatch/-/minimatch-0.2.14.tgz",
-          "integrity": "sha1-x054BXT2PG+aCQ6Q775u9TpqdWo=",
+        "param-case": {
+          "version": "2.1.1",
+          "resolved": "https://registry.npmjs.org/param-case/-/param-case-2.1.1.tgz",
+          "integrity": "sha1-35T9jPZTHs915r75oIWPvHK+Ikc=",
           "dev": true,
           "requires": {
-            "lru-cache": "2",
-            "sigmund": "~1.0.0"
+            "no-case": "2.3.2"
           }
         },
-        "rimraf": {
-          "version": "2.2.8",
-          "resolved": "https://registry.npmjs.org/rimraf/-/rimraf-2.2.8.tgz",
-          "integrity": "sha1-5Dm+Kq7jJzIZUnMPmaiSnk/FBYI=",
+        "source-map": {
+          "version": "0.6.1",
+          "resolved": "https://registry.npmjs.org/source-map/-/source-map-0.6.1.tgz",
+          "integrity": "sha512-UjgapumWlbMhkBgzT7Ykc5YXUT46F0iKu8SGXq0bcwP5dz/h0Plj6enJqjz1Zbq2l5WaqYnrVbwWOWMyF3F47g==",
           "dev": true
         },
-        "underscore": {
-          "version": "1.7.0",
-          "resolved": "https://registry.npmjs.org/underscore/-/underscore-1.7.0.tgz",
-          "integrity": "sha1-a7rwh3UA02vjTsqlhODbn+8DUgk=",
-          "dev": true
+        "uglify-js": {
+          "version": "3.4.9",
+          "resolved": "https://registry.npmjs.org/uglify-js/-/uglify-js-3.4.9.tgz",
+          "integrity": "sha512-8CJsbKOtEbnJsTyv6LE6m6ZKniqMiFWmm9sRbopbkGs3gMPPfd3Fh8iIA4Ykv5MgaTbqHr4BaoGLJLZNhsrW1Q==",
+          "dev": true,
+          "requires": {
+            "commander": "2.17.1",
+            "source-map": "0.6.1"
+          }
         }
       }
     },
-    "grunt-angular-templates": {
-      "version": "0.5.9",
-      "resolved": "https://registry.npmjs.org/grunt-angular-templates/-/grunt-angular-templates-0.5.9.tgz",
-      "integrity": "sha1-KJm+INlDitGbDQqAaqjseiOyWyo=",
-      "requires": {
-        "html-minifier": "~0.6.3"
-      }
-    },
-    "grunt-cache-bust": {
-      "version": "1.3.0",
-      "resolved": "https://registry.npmjs.org/grunt-cache-bust/-/grunt-cache-bust-1.3.0.tgz",
-      "integrity": "sha1-YtkgjiMV8cIMFgg6kHzkq8JJv1Q=",
-      "dev": true
-    },
-    "grunt-cli": {
-      "version": "0.1.13",
-      "resolved": "https://registry.npmjs.org/grunt-cli/-/grunt-cli-0.1.13.tgz",
-      "integrity": "sha1-6evEBHYx9QEtkidww5N4EzytEPQ=",
-      "dev": true,
-      "requires": {
-        "findup-sync": "~0.1.0",
-        "nopt": "~1.0.10",
-        "resolve": "~0.3.1"
-      },
-      "dependencies": {
-        "resolve": {
-          "version": "0.3.1",
-          "resolved": "https://registry.npmjs.org/resolve/-/resolve-0.3.1.tgz",
-          "integrity": "sha1-NMY0R8ZkxwWY0cmxJvxDsqJDEKQ=",
-          "dev": true
-        }
-      }
-    },
-    "grunt-concurrent": {
-      "version": "0.5.0",
-      "resolved": "https://registry.npmjs.org/grunt-concurrent/-/grunt-concurrent-0.5.0.tgz",
-      "integrity": "sha1-SlGaTCh4JfDeBxX3O4XRUMdQ2fc=",
-      "dev": true,
-      "requires": {
-        "async": "~0.2.9",
-        "pad-stdio": "^0.1.0"
-      }
-    },
-    "grunt-contrib-concat": {
-      "version": "0.4.0",
-      "resolved": "https://registry.npmjs.org/grunt-contrib-concat/-/grunt-contrib-concat-0.4.0.tgz",
-      "integrity": "sha1-uH988VO/ZGiBQvlHFhFWAT+8fHQ=",
+    "htmlhint": {
+      "version": "0.9.13",
+      "resolved": "https://registry.npmjs.org/htmlhint/-/htmlhint-0.9.13.tgz",
+      "integrity": "sha1-CBY8seaqUFBI67C0EGOnygfcbIg=",
       "dev": true,
       "requires": {
-        "chalk": "~0.4.0"
+        "async": "1.4.2",
+        "colors": "1.0.3",
+        "commander": "2.6.0",
+        "csslint": "0.10.0",
+        "glob": "5.0.15",
+        "jshint": "2.8.0",
+        "parse-glob": "3.0.4",
+        "strip-json-comments": "1.0.4",
+        "xml": "1.0.0"
       },
       "dependencies": {
-        "ansi-styles": {
-          "version": "1.0.0",
-          "resolved": "https://registry.npmjs.org/ansi-styles/-/ansi-styles-1.0.0.tgz",
-          "integrity": "sha1-yxAt8cVvUSPquLZ817mAJ6AnkXg=",
+        "async": {
+          "version": "1.4.2",
+          "resolved": "https://registry.npmjs.org/async/-/async-1.4.2.tgz",
+          "integrity": "sha1-bJ7csRztTw3S8tQNsNSaEJwIiqs=",
           "dev": true
         },
-        "chalk": {
-          "version": "0.4.0",
-          "resolved": "https://registry.npmjs.org/chalk/-/chalk-0.4.0.tgz",
-          "integrity": "sha1-UZmj3c0MHv4jvAjBsCewYXbgxk8=",
-          "dev": true,
-          "requires": {
-            "ansi-styles": "~1.0.0",
-            "has-color": "~0.1.0",
-            "strip-ansi": "~0.1.0"
-          }
-        },
-        "strip-ansi": {
-          "version": "0.1.1",
-          "resolved": "https://registry.npmjs.org/strip-ansi/-/strip-ansi-0.1.1.tgz",
-          "integrity": "sha1-OeipjQRNFQZgq+SmgIrPcLt7yZE=",
+        "colors": {
+          "version": "1.0.3",
+          "resolved": "https://registry.npmjs.org/colors/-/colors-1.0.3.tgz",
+          "integrity": "sha1-BDP0TYCWgP3rYO0mDxsMJi6CpAs=",
           "dev": true
-        }
-      }
-    },
-    "grunt-contrib-copy": {
-      "version": "0.5.0",
-      "resolved": "https://registry.npmjs.org/grunt-contrib-copy/-/grunt-contrib-copy-0.5.0.tgz",
-      "integrity": "sha1-QQB1rEWlhWuhkbHMclclRQ1KAhU=",
-      "dev": true
-    },
-    "grunt-contrib-cssmin": {
-      "version": "0.9.0",
-      "resolved": "https://registry.npmjs.org/grunt-contrib-cssmin/-/grunt-contrib-cssmin-0.9.0.tgz",
-      "integrity": "sha1-JyQfAWCohmZZ2rQNyMJ3bAHsfOI=",
-      "dev": true,
-      "requires": {
-        "chalk": "~0.4.0",
-        "clean-css": "~2.1.0",
-        "maxmin": "~0.1.0"
-      },
-      "dependencies": {
-        "ansi-styles": {
-          "version": "1.0.0",
-          "resolved": "https://registry.npmjs.org/ansi-styles/-/ansi-styles-1.0.0.tgz",
-          "integrity": "sha1-yxAt8cVvUSPquLZ817mAJ6AnkXg=",
+        },
+        "commander": {
+          "version": "2.6.0",
+          "resolved": "https://registry.npmjs.org/commander/-/commander-2.6.0.tgz",
+          "integrity": "sha1-nfflL7Kgyw+4kFjugMMQQiXzfh0=",
           "dev": true
         },
-        "chalk": {
-          "version": "0.4.0",
-          "resolved": "https://registry.npmjs.org/chalk/-/chalk-0.4.0.tgz",
-          "integrity": "sha1-UZmj3c0MHv4jvAjBsCewYXbgxk8=",
+        "glob": {
+          "version": "5.0.15",
+          "resolved": "https://registry.npmjs.org/glob/-/glob-5.0.15.tgz",
+          "integrity": "sha1-G8k2ueAvSmA/zCIuz3Yz0wuLk7E=",
           "dev": true,
           "requires": {
-            "ansi-styles": "~1.0.0",
-            "has-color": "~0.1.0",
-            "strip-ansi": "~0.1.0"
+            "inflight": "1.0.6",
+            "inherits": "2.0.3",
+            "minimatch": "3.0.4",
+            "once": "1.4.0",
+            "path-is-absolute": "1.0.1"
           }
         },
-        "clean-css": {
-          "version": "2.1.8",
-          "resolved": "https://registry.npmjs.org/clean-css/-/clean-css-2.1.8.tgz",
-          "integrity": "sha1-K0sv1g8yRBCWIWriWiH6p0WA3IM=",
+        "minimatch": {
+          "version": "3.0.4",
+          "resolved": "https://registry.npmjs.org/minimatch/-/minimatch-3.0.4.tgz",
+          "integrity": "sha512-yJHVQEhyqPLUTgt9B83PXu6W3rx4MvvHvSUvToogpwoGDOUQ+yDrR0HRot+yOCdCO7u4hX3pWft6kWBBcqh0UA==",
           "dev": true,
           "requires": {
-            "commander": "2.1.x"
+            "brace-expansion": "1.1.11"
           }
         },
-        "commander": {
-          "version": "2.1.0",
-          "resolved": "https://registry.npmjs.org/commander/-/commander-2.1.0.tgz",
-          "integrity": "sha1-0SG7roYNmZKj1Re6lvVliOR8Z4E=",
-          "dev": true
-        },
-        "strip-ansi": {
-          "version": "0.1.1",
-          "resolved": "https://registry.npmjs.org/strip-ansi/-/strip-ansi-0.1.1.tgz",
-          "integrity": "sha1-OeipjQRNFQZgq+SmgIrPcLt7yZE=",
+        "strip-json-comments": {
+          "version": "1.0.4",
+          "resolved": "https://registry.npmjs.org/strip-json-comments/-/strip-json-comments-1.0.4.tgz",
+          "integrity": "sha1-HhX7ysl9Pumb8tc7TGVrCCu6+5E=",
           "dev": true
         }
       }
     },
-    "grunt-contrib-htmlmin": {
-      "version": "0.3.0",
-      "resolved": "https://registry.npmjs.org/grunt-contrib-htmlmin/-/grunt-contrib-htmlmin-0.3.0.tgz",
-      "integrity": "sha1-yWCAIEj2CZJenQ7xsGcJBLTFo/0=",
-      "dev": true,
+    "htmlparser2": {
+      "version": "3.1.4",
+      "resolved": "https://registry.npmjs.org/htmlparser2/-/htmlparser2-3.1.4.tgz",
+      "integrity": "sha1-csvn1dVsAaz2H897kzMx9ORbNvA=",
       "requires": {
-        "chalk": "~0.4.0",
-        "html-minifier": "~0.6.0",
-        "pretty-bytes": "~0.1.0"
+        "domelementtype": "1.2.1",
+        "domhandler": "2.0.3",
+        "domutils": "1.1.6",
+        "readable-stream": "1.0.34"
       },
       "dependencies": {
-        "ansi-styles": {
-          "version": "1.0.0",
-          "resolved": "https://registry.npmjs.org/ansi-styles/-/ansi-styles-1.0.0.tgz",
-          "integrity": "sha1-yxAt8cVvUSPquLZ817mAJ6AnkXg=",
-          "dev": true
-        },
-        "chalk": {
-          "version": "0.4.0",
-          "resolved": "https://registry.npmjs.org/chalk/-/chalk-0.4.0.tgz",
-          "integrity": "sha1-UZmj3c0MHv4jvAjBsCewYXbgxk8=",
-          "dev": true,
+        "domutils": {
+          "version": "1.1.6",
+          "resolved": "https://registry.npmjs.org/domutils/-/domutils-1.1.6.tgz",
+          "integrity": "sha1-vdw94Jm5ou+sxRxiPyj0FuzFdIU=",
           "requires": {
-            "ansi-styles": "~1.0.0",
-            "has-color": "~0.1.0",
-            "strip-ansi": "~0.1.0"
+            "domelementtype": "1.2.1"
           }
-        },
-        "strip-ansi": {
-          "version": "0.1.1",
-          "resolved": "https://registry.npmjs.org/strip-ansi/-/strip-ansi-0.1.1.tgz",
-          "integrity": "sha1-OeipjQRNFQZgq+SmgIrPcLt7yZE=",
-          "dev": true
         }
       }
     },
-    "grunt-contrib-uglify": {
-      "version": "0.4.1",
-      "resolved": "https://registry.npmjs.org/grunt-contrib-uglify/-/grunt-contrib-uglify-0.4.1.tgz",
-      "integrity": "sha1-1D87xuAsM1Vj+MT58IE/tLD/ebE=",
-      "dev": true,
-      "requires": {
-        "chalk": "^0.4.0",
-        "maxmin": "^0.1.0",
-        "uglify-js": "^2.4.0"
-      },
-      "dependencies": {
-        "ansi-styles": {
-          "version": "1.0.0",
-          "resolved": "https://registry.npmjs.org/ansi-styles/-/ansi-styles-1.0.0.tgz",
-          "integrity": "sha1-yxAt8cVvUSPquLZ817mAJ6AnkXg=",
-          "dev": true
-        },
-        "chalk": {
-          "version": "0.4.0",
-          "resolved": "https://registry.npmjs.org/chalk/-/chalk-0.4.0.tgz",
-          "integrity": "sha1-UZmj3c0MHv4jvAjBsCewYXbgxk8=",
-          "dev": true,
-          "requires": {
-            "ansi-styles": "~1.0.0",
-            "has-color": "~0.1.0",
-            "strip-ansi": "~0.1.0"
-          }
-        },
-        "strip-ansi": {
-          "version": "0.1.1",
-          "resolved": "https://registry.npmjs.org/strip-ansi/-/strip-ansi-0.1.1.tgz",
-          "integrity": "sha1-OeipjQRNFQZgq+SmgIrPcLt7yZE=",
-          "dev": true
-        }
-      }
+    "http-deceiver": {
+      "version": "1.2.7",
+      "resolved": "https://registry.npmjs.org/http-deceiver/-/http-deceiver-1.2.7.tgz",
+      "integrity": "sha1-+nFolEq5pRnTN8sL7HKE3D5yPYc=",
+      "dev": true
     },
-    "grunt-contrib-watch": {
-      "version": "0.6.1",
-      "resolved": "https://registry.npmjs.org/grunt-contrib-watch/-/grunt-contrib-watch-0.6.1.tgz",
-      "integrity": "sha1-ZP3LolpjX1tNobbOb5DaCutuPxU=",
+    "http-errors": {
+      "version": "1.6.3",
+      "resolved": "https://registry.npmjs.org/http-errors/-/http-errors-1.6.3.tgz",
+      "integrity": "sha1-i1VoC7S+KDoLW/TqLjhYC+HZMg0=",
       "dev": true,
       "requires": {
-        "async": "~0.2.9",
-        "gaze": "~0.5.1",
-        "lodash": "~2.4.1",
-        "tiny-lr-fork": "0.0.5"
-      },
-      "dependencies": {
-        "lodash": {
-          "version": "2.4.2",
-          "resolved": "https://registry.npmjs.org/lodash/-/lodash-2.4.2.tgz",
-          "integrity": "sha1-+t2DS5aDBz2hebPq5tnA0VBT9z4=",
-          "dev": true
-        }
+        "depd": "1.1.2",
+        "inherits": "2.0.3",
+        "setprototypeof": "1.1.0",
+        "statuses": "1.4.0"
       }
     },
-    "grunt-dom-munger": {
-      "version": "3.4.0",
-      "resolved": "https://registry.npmjs.org/grunt-dom-munger/-/grunt-dom-munger-3.4.0.tgz",
-      "integrity": "sha1-LQ2Plk9amVEekUrR1T8fccWrbYk=",
+    "http-parser-js": {
+      "version": "0.5.0",
+      "resolved": "https://registry.npmjs.org/http-parser-js/-/http-parser-js-0.5.0.tgz",
+      "integrity": "sha512-cZdEF7r4gfRIq7ezX9J0T+kQmJNOub71dWbgAXVHDct80TKP4MCETtZQ31xyv38UwgzkWPYF/Xc0ge55dW9Z9w==",
+      "dev": true
+    },
+    "http-proxy": {
+      "version": "1.17.0",
+      "resolved": "https://registry.npmjs.org/http-proxy/-/http-proxy-1.17.0.tgz",
+      "integrity": "sha512-Taqn+3nNvYRfJ3bGvKfBSRwy1v6eePlm3oc/aWVxZp57DQr5Eq3xhKJi7Z4hZpS8PC3H4qI+Yly5EmFacGuA/g==",
+      "dev": true,
       "requires": {
-        "cheerio": "~0.12.3"
+        "eventemitter3": "3.1.0",
+        "follow-redirects": "1.5.9",
+        "requires-port": "1.0.0"
       }
     },
-    "grunt-filerev": {
-      "version": "0.2.1",
-      "resolved": "https://registry.npmjs.org/grunt-filerev/-/grunt-filerev-0.2.1.tgz",
-      "integrity": "sha1-Svngz+2nuwFnB2VpeREimBH29NM=",
+    "http-proxy-middleware": {
+      "version": "0.18.0",
+      "resolved": "https://registry.npmjs.org/http-proxy-middleware/-/http-proxy-middleware-0.18.0.tgz",
+      "integrity": "sha512-Fs25KVMPAIIcgjMZkVHJoKg9VcXcC1C8yb9JUgeDvVXY0S/zgVIhMb+qVswDIgtJe2DfckMSY2d6TuTEutlk6Q==",
       "dev": true,
       "requires": {
-        "chalk": "~0.4.0",
-        "each-async": "~0.1.0"
+        "http-proxy": "1.17.0",
+        "is-glob": "4.0.0",
+        "lodash": "4.17.11",
+        "micromatch": "3.1.10"
       },
       "dependencies": {
-        "ansi-styles": {
-          "version": "1.0.0",
-          "resolved": "https://registry.npmjs.org/ansi-styles/-/ansi-styles-1.0.0.tgz",
-          "integrity": "sha1-yxAt8cVvUSPquLZ817mAJ6AnkXg=",
-          "dev": true
-        },
-        "chalk": {
-          "version": "0.4.0",
-          "resolved": "https://registry.npmjs.org/chalk/-/chalk-0.4.0.tgz",
-          "integrity": "sha1-UZmj3c0MHv4jvAjBsCewYXbgxk8=",
-          "dev": true,
-          "requires": {
-            "ansi-styles": "~1.0.0",
-            "has-color": "~0.1.0",
-            "strip-ansi": "~0.1.0"
-          }
-        },
-        "strip-ansi": {
-          "version": "0.1.1",
-          "resolved": "https://registry.npmjs.org/strip-ansi/-/strip-ansi-0.1.1.tgz",
-          "integrity": "sha1-OeipjQRNFQZgq+SmgIrPcLt7yZE=",
-          "dev": true
-        }
-      }
-    },
-    "grunt-htmlhint": {
-      "version": "0.9.13",
-      "resolved": "https://registry.npmjs.org/grunt-htmlhint/-/grunt-htmlhint-0.9.13.tgz",
-      "integrity": "sha1-cXACPzDi5wUnkjQrSNW7+RK512w=",
-      "dev": true,
-      "requires": {
-        "htmlhint": "~0.9.13"
-      }
-    },
-    "grunt-legacy-log": {
-      "version": "0.1.3",
-      "resolved": "https://registry.npmjs.org/grunt-legacy-log/-/grunt-legacy-log-0.1.3.tgz",
-      "integrity": "sha1-7ClCboAwIa9ZAp+H0vnNczWgVTE=",
-      "dev": true,
-      "requires": {
-        "colors": "~0.6.2",
-        "grunt-legacy-log-utils": "~0.1.1",
-        "hooker": "~0.2.3",
-        "lodash": "~2.4.1",
-        "underscore.string": "~2.3.3"
-      },
-      "dependencies": {
-        "colors": {
-          "version": "0.6.2",
-          "resolved": "https://registry.npmjs.org/colors/-/colors-0.6.2.tgz",
-          "integrity": "sha1-JCP+ZnisDF2uiFLl0OW+CMmXq8w=",
-          "dev": true
-        },
-        "lodash": {
-          "version": "2.4.2",
-          "resolved": "https://registry.npmjs.org/lodash/-/lodash-2.4.2.tgz",
-          "integrity": "sha1-+t2DS5aDBz2hebPq5tnA0VBT9z4=",
-          "dev": true
-        },
-        "underscore.string": {
-          "version": "2.3.3",
-          "resolved": "https://registry.npmjs.org/underscore.string/-/underscore.string-2.3.3.tgz",
-          "integrity": "sha1-ccCL9rQosRM/N+ePo6Icgvcymw0=",
-          "dev": true
-        }
-      }
-    },
-    "grunt-legacy-log-utils": {
-      "version": "0.1.1",
-      "resolved": "https://registry.npmjs.org/grunt-legacy-log-utils/-/grunt-legacy-log-utils-0.1.1.tgz",
-      "integrity": "sha1-wHBrndkGThFvNvI/5OawSGcsD34=",
-      "dev": true,
-      "requires": {
-        "colors": "~0.6.2",
-        "lodash": "~2.4.1",
-        "underscore.string": "~2.3.3"
-      },
-      "dependencies": {
-        "colors": {
-          "version": "0.6.2",
-          "resolved": "https://registry.npmjs.org/colors/-/colors-0.6.2.tgz",
-          "integrity": "sha1-JCP+ZnisDF2uiFLl0OW+CMmXq8w=",
-          "dev": true
-        },
-        "lodash": {
-          "version": "2.4.2",
-          "resolved": "https://registry.npmjs.org/lodash/-/lodash-2.4.2.tgz",
-          "integrity": "sha1-+t2DS5aDBz2hebPq5tnA0VBT9z4=",
-          "dev": true
-        },
-        "underscore.string": {
-          "version": "2.3.3",
-          "resolved": "https://registry.npmjs.org/underscore.string/-/underscore.string-2.3.3.tgz",
-          "integrity": "sha1-ccCL9rQosRM/N+ePo6Icgvcymw0=",
-          "dev": true
-        }
-      }
-    },
-    "grunt-legacy-util": {
-      "version": "0.2.0",
-      "resolved": "https://registry.npmjs.org/grunt-legacy-util/-/grunt-legacy-util-0.2.0.tgz",
-      "integrity": "sha1-kzJIhNv343qf98Am3/RR2UqeVUs=",
-      "dev": true,
-      "requires": {
-        "async": "~0.1.22",
-        "exit": "~0.1.1",
-        "getobject": "~0.1.0",
-        "hooker": "~0.2.3",
-        "lodash": "~0.9.2",
-        "underscore.string": "~2.2.1",
-        "which": "~1.0.5"
-      },
-      "dependencies": {
-        "async": {
-          "version": "0.1.22",
-          "resolved": "https://registry.npmjs.org/async/-/async-0.1.22.tgz",
-          "integrity": "sha1-D8GqoIig4+8Ovi2IMbqw3PiEUGE=",
-          "dev": true
-        },
-        "lodash": {
-          "version": "0.9.2",
-          "resolved": "https://registry.npmjs.org/lodash/-/lodash-0.9.2.tgz",
-          "integrity": "sha1-jzSZxSRdNG1oLlsNO0B2fgnxqSw=",
-          "dev": true
-        }
-      }
-    },
-    "grunt-newer": {
-      "version": "0.7.0",
-      "resolved": "https://registry.npmjs.org/grunt-newer/-/grunt-newer-0.7.0.tgz",
-      "integrity": "sha1-N22dm2TOXGSLa/ob2pj3vCGT5B4=",
-      "dev": true,
-      "requires": {
-        "async": "0.2.10",
-        "rimraf": "2.2.6"
-      },
-      "dependencies": {
-        "rimraf": {
-          "version": "2.2.6",
-          "resolved": "https://registry.npmjs.org/rimraf/-/rimraf-2.2.6.tgz",
-          "integrity": "sha1-xZWXVpsU2VatKcrMQr3d9fDqT0w=",
-          "dev": true
-        }
-      }
-    },
... 10935 lines suppressed ...