You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@zeppelin.apache.org by li...@apache.org on 2019/06/21 02:05:05 UTC

[zeppelin] branch master updated: [ZEPPELIN-4104] Zeppelin Interpreter On Docker

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

liuxun 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 c8a82bc  [ZEPPELIN-4104] Zeppelin Interpreter On Docker
c8a82bc is described below

commit c8a82bce2bc9fe6a98f99824bdbddcc06b054742
Author: Xun Liu <li...@apache.org>
AuthorDate: Sat Jun 15 10:06:46 2019 +0800

    [ZEPPELIN-4104] Zeppelin Interpreter On Docker
    
    ### What is this PR for?
    Zeppelin service runs on local server. zeppelin is able to run the interpreter in the docker container, Isolating the operating environment of the interpreter through the docker container. Zeppelin can be easily used without having to install python, spark, etc. on the local node.
    
    Design Doc : https://docs.google.com/document/d/1ESQEHmGlw4kaXkK5Hn2AOJYBLWDZRU9shB3WHuGV-CM/edit
    
    #### Key benefits are
    1. Interpreter environment isolating
    2. Not need to install `python lib`, `spark binary package`, etc. environment on the local node
    3. Docker does not need to pre-install zeppelin binary package, Automatically upload local zeppelin interpreter lib files to the container
    4. Automatically upload local configuration files (such as `spark-conf`, `hadoop-conf-dir`, `keytab file`, ...) to the container, so that the running environment in the container is exactly the same as the local.
    5. Zeppelin server runs locally, making it easier to manage and maintain
    6. Support zeppelin all interpreter
    7. Support spark interpreter, `local[*]`, `cluster-client`, `cluster-yarn` mode.
    
    ### What type of PR is it?
    [Feature]
    
    ### What is the Jira issue?
    * https://issues.apache.org/jira/browse/ZEPPELIN-4104
    
    ### How should this be tested?
    * CI Pass
    
    ### Screenshots
    
    ### Sample1 : Spark(yarn-cluster mode) interpreter run docker
    ![sparkOnDocker1](https://user-images.githubusercontent.com/3677382/59410510-9a799100-8deb-11e9-9cb2-658ff3d4f552.gif)
    
    ### Sample2 : Spark(yarn-cluster mode) interpreter run docker
    ![sparkOnDocker2](https://user-images.githubusercontent.com/3677382/59410520-9ea5ae80-8deb-11e9-83ff-330cde39b15e.gif)
    
    ### Sample3 : Shell interpreter run docker
    ![shellOnDocker-800](https://user-images.githubusercontent.com/3677382/59410410-58e8e600-8deb-11e9-8948-531c2a8bea61.gif)
    
    ### Questions:
    * Does the licenses files need update? NO
    * Is there breaking changes for older versions? No
    * Does this needs documentation? YES
    
    Author: Xun Liu <li...@apache.org>
    
    Closes #3383 from liuxunorg/ZEPPELIN-4104 and squashes the following commits:
    
    a16bb71f9 [Xun Liu] Solve CI problem : Line is longer than 100 characters.
    2229b90ae [Xun Liu] Modify getZeppelinHome()
    e4bef4059 [Xun Liu] Modify the connectTimeout log content.
    1b272d9d4 [Xun Liu] Restore the zeppelin-plugins pom
    ce4e3128b [Xun Liu] [ZEPPELIN-4104] Zeppelin Interpreter On Docker
---
 conf/zeppelin-env.sh.template                      |   2 +-
 conf/zeppelin-site.xml.template                    |   8 +-
 docs/_includes/themes/zeppelin/_navigation.html    |   5 +-
 docs/quickstart/docker.md                          | 199 +++++++
 scripts/docker/interpreter/Dockerfile              |  32 ++
 .../zeppelin/conf/ZeppelinConfiguration.java       |  11 +-
 .../interpreter/launcher/InterpreterLauncher.java  |   4 +
 zeppelin-plugins/launcher/docker/pom.xml           | 116 +++++
 .../launcher/DockerInterpreterLauncher.java        |  80 +++
 .../launcher/DockerInterpreterProcess.java         | 576 +++++++++++++++++++++
 .../interpreter/launcher/DockerSpecTemplate.java   |  78 +++
 .../interpreter/launcher/utils/TarFileEntry.java   |  37 ++
 .../interpreter/launcher/utils/TarUtils.java       | 103 ++++
 .../jinja_templates/docker-interpreter.jinja       |  17 +
 .../launcher/DockerInterpreterProcessTest.java     | 149 ++++++
 zeppelin-plugins/pom.xml                           |   1 +
 .../zeppelin/interpreter/InterpreterSetting.java   |   7 +
 17 files changed, 1420 insertions(+), 5 deletions(-)

diff --git a/conf/zeppelin-env.sh.template b/conf/zeppelin-env.sh.template
index 74941b9..d8e265a 100644
--- a/conf/zeppelin-env.sh.template
+++ b/conf/zeppelin-env.sh.template
@@ -72,6 +72,7 @@
 # export SPARK_HOME                             # (required) When it is defined, load it instead of Zeppelin embedded Spark libraries
 # export SPARK_SUBMIT_OPTIONS                   # (optional) extra options to pass to spark submit. eg) "--driver-memory 512M --executor-memory 1G".
 # export SPARK_APP_NAME                         # (optional) The name of spark application.
+# export SPARK_CONF_DIR                         # (optional) In the zeppelin interpreter on docker mode, Need to set the local spark conf folder path
 
 ## Use embedded spark binaries ##
 ## without SPARK_HOME defined, Zeppelin still able to run spark interpreter process using embedded spark binaries.
@@ -92,7 +93,6 @@
 # export ZEPPELIN_SPARK_MAXRESULT       # Max number of Spark SQL result to display. 1000 by default.
 # export ZEPPELIN_WEBSOCKET_MAX_TEXT_MESSAGE_SIZE       # Size in characters of the maximum text message to be received by websocket. Defaults to 1024000
 
-
 #### HBase interpreter configuration ####
 
 ## To connect to HBase running on a cluster, either HBASE_HOME or HBASE_CONF_DIR must be set
diff --git a/conf/zeppelin-site.xml.template b/conf/zeppelin-site.xml.template
index 8115f7b..e53790d 100755
--- a/conf/zeppelin-site.xml.template
+++ b/conf/zeppelin-site.xml.template
@@ -610,7 +610,7 @@
 <property>
   <name>zeppelin.run.mode</name>
   <value>auto</value>
-  <description>'auto|local|k8s'</description>
+  <description>'auto|local|k8s|docker'</description>
 </property>
 
 <property>
@@ -637,4 +637,10 @@
   <description>Kubernetes yaml spec files</description>
 </property>
 
+  <property>
+    <name>zeppelin.docker.container.image</name>
+    <value>apache/zeppelin:0.8.0</value>
+    <description>Docker image for interpreters</description>
+  </property>
+
 </configuration>
diff --git a/docs/_includes/themes/zeppelin/_navigation.html b/docs/_includes/themes/zeppelin/_navigation.html
index 8ada4e9..4858691 100644
--- a/docs/_includes/themes/zeppelin/_navigation.html
+++ b/docs/_includes/themes/zeppelin/_navigation.html
@@ -25,10 +25,13 @@
               <ul class="dropdown-menu">
                 <li class="title"><span>Getting Started</span></li>
                 <li><a href="{{BASE_PATH}}/quickstart/install.html">Install</a></li>
-                <li><a href="{{BASE_PATH}}/quickstart/kubernetes.html">Kubernetes</a></li>
                 <li><a href="{{BASE_PATH}}/quickstart/explore_ui.html">Explore UI</a></li>
                 <li><a href="{{BASE_PATH}}/quickstart/tutorial.html">Tutorial</a></li>
                 <li role="separator" class="divider"></li>
+                <li class="title"><span>Run Mode</span></li>
+                <li><a href="{{BASE_PATH}}/quickstart/kubernetes.html">Kubernetes</a></li>
+                <li><a href="{{BASE_PATH}}/quickstart/docker.html">Docker</a></li>
+                <li role="separator" class="divider"></li>
                 <li><a href="{{BASE_PATH}}/quickstart/spark_with_zeppelin.html">Spark with Zeppelin</a></li>
                 <li><a href="{{BASE_PATH}}/quickstart/sql_with_zeppelin.html">SQL with Zeppelin</a></li>
                 <li><a href="{{BASE_PATH}}/quickstart/python_with_zeppelin.html">Python with Zeppelin</a></li>
diff --git a/docs/quickstart/docker.md b/docs/quickstart/docker.md
new file mode 100644
index 0000000..9cb2c4f
--- /dev/null
+++ b/docs/quickstart/docker.md
@@ -0,0 +1,199 @@
+---
+layout: page
+title: "Install"
+description: "This page will help you get started and will guide you through installing Apache Zeppelin and running it in the command line."
+group: quickstart
+---
+<!--
+Licensed under the Apache License, Version 2.0 (the "License");
+you may not use this file except in compliance with the License.
+You may obtain a copy of the License at
+
+http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS,
+WITHOUT 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 %}
+
+# Zeppelin interpreter on Docker
+
+Zeppelin service runs on local server. zeppelin is able to run the interpreter in the docker container, Isolating the operating environment of the interpreter through the docker container. Zeppelin can be easily used without having to install python, spark, etc. on the local node.
+
+Key benefits are
+
+ - Interpreter environment isolating
+ - Not need to install python, spark, etc. environment on the local node
+ - Docker does not need to pre-install zeppelin binary package, Automatically upload local zeppelin interpreter lib files to the container
+ - Automatically upload local configuration files (such as spark-conf, hadoop-conf-dir, keytab file, ...) to the container, so that the running environment in the container is exactly the same as the local.
+ - Zeppelin server runs locally, making it easier to manage and maintain
+
+## Prerequisites
+
+ - apache/zeppelin docker image
+ - Spark >= 2.2.0 docker image (in case of using Spark Interpreter)
+ - Docker 1.6+ [Install Docker](https://docs.docker.com/v17.12/install/)
+ - Use docker's host network, so there is no need to set up a network specifically
+
+### Docker Configuration
+
+Because `DockerInterpreterProcess` communicates via docker's tcp interface.
+
+By default, docker provides an interface as a sock file, so you need to modify the configuration file to open the tcp interface remotely.
+
+vi `/etc/docker/daemon.json`, Add `tcp://0.0.0.0:2375` to the `hosts` configuration item.
+
+```
+{
+    ...
+    "hosts": ["tcp://0.0.0.0:2375","unix:///var/run/docker.sock"]
+}
+```
+
+`hosts` property reference: https://docs.docker.com/engine/reference/commandline/dockerd/
+
+
+## Quickstart
+
+Modify these 2 configuration items in `zeppelin-site.xml`.
+
+```
+  <property>
+    <name>zeppelin.run.mode</name>
+    <value>docker</value>
+    <description>'auto|local|k8s|docker'</description>
+  </property>
+
+  <property>
+    <name>zeppelin.docker.container.image</name>
+    <value>apache/zeppelin</value>
+    <description>Docker image for interpreters</description>
+  </property>
+
+```
+
+
+## Build Zeppelin image manually
+
+To build Zeppelin image, support Kerberos certification & install spark binary.
+
+Use the `/scripts/docker/interpreter/Dockerfile` to build the image.
+
+```
+FROM apache/zeppelin:0.8.0
+MAINTAINER Apache Software Foundation <de...@zeppelin.apache.org>
+
+ENV SPARK_VERSION=2.3.3
+ENV HADOOP_VERSION=2.7
+
+# support Kerberos certification
+RUN install -yq curl unzip wget grep sed vim krb5-user libpam-krb5 && apt-get clean
+
+# auto upload zeppelin interpreter lib
+RUN rm /zeppelin -R
+
+RUN rm /spark -R
+RUN wget https://www-us.apache.org/dist/spark/spark-${SPARK_VERSION}/spark-${SPARK_VERSION}-bin-hadoop${HADOOP_VERSION}.tgz
+RUN tar zxvf spark-${SPARK_VERSION}-bin-hadoop${HADOOP_VERSION}.tgz
+RUN mv spark-${SPARK_VERSION}-bin-hadoop${HADOOP_VERSION} spark
+RUN rm spark-${SPARK_VERSION}-bin-hadoop${HADOOP_VERSION}.tgz
+```
+
+Then build docker image.
+
+```
+# build image. Replace <tag>.
+$ docker build -t <tag> .
+```
+
+## How it works
+
+### Zeppelin interpreter on Docker
+
+Zeppelin service runs on local server, it auto configure itself to use `DockerInterpreterLauncher`.
+
+`DockerInterpreterLauncher` via `DockerInterpreterProcess` launcher creates each interpreter in a container using docker image.
+
+`DockerInterpreterProcess` uploads the binaries and configuration files of the local zeppelin service to the container:
+
+ - ${ZEPPELIN_HOME}/bin
+ - ${ZEPPELIN_HOME}/lib
+ - ${ZEPPELIN_HOME}/interpreter/${interpreterGroupName}
+ - ${ZEPPELIN_HOME}/conf/zeppelin-site.xml
+ - ${ZEPPELIN_HOME}/conf/log4j.properties
+ - ${ZEPPELIN_HOME}/conf/log4j_yarn_cluster.properties
+ - HADOOP_CONF_DIR
+ - SPARK_CONF_DIR
+ - /etc/krb5.conf
+ - Keytab file configured in the interpreter properties
+   - zeppelin.shell.keytab.location
+   - spark.yarn.keytab
+   - submarine.hadoop.keytab
+   - zeppelin.jdbc.keytab.location
+   - zeppelin.server.kerberos.keytab
+
+All file paths uploaded to the container, Keep the same path as the local one. This will ensure that all configurations are used correctly.
+
+### Spark interpreter on Docker
+
+When interpreter group is `spark`, Zeppelin sets necessary spark configuration automatically to use Spark on Docker.
+Supports all running modes of `local[*]`, `yarn-client`, and `yarn-cluster` of zeppelin spark interpreter.
+
+#### SPARK_CONF_DIR
+
+1. Configuring in the zeppelin-env.sh
+
+  Because there are only spark binary files in the interpreter image, no spark conf files are included.
+  The configuration file in the `spark-<version>/conf/` local to the zeppelin service needs to be uploaded to the ``/spark/conf/` directory in the spark interpreter container.
+  So you need to setting `export SPARK_CONF_DIR=/spark-<version>-path/conf/` in the `zeppelin-env.sh` file.
+
+2. Configuring in the spark Properties
+
+  You can also configure it in the spark interpreter properties.
+
+  | properties name | Value | Description |
+  | ----- | ----- | ----- |
+  | SPARK_CONF_DIR | /spark-<version>-path.../conf/ | Spark-<version>-path/conf/ path local on the zeppelin service |
+
+
+#### HADOOP_CONF_DIR
+
+1. Configuring in the zeppelin-env.sh
+
+  Because there are only spark binary files in the interpreter image, no configuration files are included.
+  The configuration file in the `hadoop-<version>/etc/hadoop` local to the zeppelin service needs to be uploaded to the spark interpreter container.
+  So you need to setting `export HADOOP_CONF_DIR=hadoop-<version>-path/etc/hadoop` in the `zeppelin-env.sh` file.
+
+2. Configuring in the spark Properties
+
+  You can also configure it in the spark interpreter properties.
+
+  | properties name | Value | Description |
+  | ----- | ----- | ----- |
+  | HADOOP_CONF_DIR | hadoop-<version>-path/etc/hadoop | hadoop-<version>-path/etc/hadoop path local on the zeppelin service |
+
+
+#### Accessing Spark UI (or Service running in interpreter container)
+
+Because the zeppelin interpreter container uses the host network, the spark.ui.port port is automatically allocated, so do not configure `spark.ui.port=xxxx` in `spark-defaults.conf`
+
+
+## Future work
+
+ - Configuring container resources that can be used by different interpreters by configuration.
+
+
+## Development
+
+Instead of build Zeppelin distribution package and docker image everytime during development,
+Zeppelin can run locally (such as inside your IDE in debug mode) and able to run Interpreter using [DockerInterpreterLauncher](https://github.com/apache/zeppelin/blob/master/zeppelin-plugins/launcher/docker/src/main/java/org/apache/zeppelin/interpreter/launcher/DockerInterpreterLauncher.java) by configuring following environment variables.
+
+
+| Environment variable | Value | Description |
+| ----- | ----- | ----- |
+| ZEPPELIN_RUN_MODE | docker | Make Zeppelin run interpreter on Docker |
+| ZEPPELIN_DOCKER_CONTAINER_IMAGE | <image>:<version> | Zeppelin interpreter docker image to use |
+
diff --git a/scripts/docker/interpreter/Dockerfile b/scripts/docker/interpreter/Dockerfile
new file mode 100644
index 0000000..ca282ee
--- /dev/null
+++ b/scripts/docker/interpreter/Dockerfile
@@ -0,0 +1,32 @@
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+FROM apache/zeppelin:0.8.0
+MAINTAINER Apache Software Foundation <de...@zeppelin.apache.org>
+
+ENV SPARK_VERSION=2.3.3
+ENV HADOOP_VERSION=2.7
+
+# support Kerberos certification
+RUN install -yq curl unzip wget grep sed vim krb5-user libpam-krb5 && apt-get clean
+
+# auto upload zeppelin interpreter lib
+RUN rm /zeppelin -R
+
+RUN rm /spark -R
+RUN wget https://www-us.apache.org/dist/spark/spark-${SPARK_VERSION}/spark-${SPARK_VERSION}-bin-hadoop${HADOOP_VERSION}.tgz
+RUN tar zxvf spark-${SPARK_VERSION}-bin-hadoop${HADOOP_VERSION}.tgz
+RUN mv spark-${SPARK_VERSION}-bin-hadoop${HADOOP_VERSION} spark
+RUN rm spark-${SPARK_VERSION}-bin-hadoop${HADOOP_VERSION}.tgz
diff --git a/zeppelin-interpreter/src/main/java/org/apache/zeppelin/conf/ZeppelinConfiguration.java b/zeppelin-interpreter/src/main/java/org/apache/zeppelin/conf/ZeppelinConfiguration.java
index d4824dc..e16ade8 100644
--- a/zeppelin-interpreter/src/main/java/org/apache/zeppelin/conf/ZeppelinConfiguration.java
+++ b/zeppelin-interpreter/src/main/java/org/apache/zeppelin/conf/ZeppelinConfiguration.java
@@ -52,7 +52,8 @@ public class ZeppelinConfiguration extends XMLConfiguration {
 
   public enum RUN_MODE {
     LOCAL,
-    K8S
+    K8S,
+    DOCKER
   }
 
   public ZeppelinConfiguration(URL url) throws ConfigurationException {
@@ -735,6 +736,10 @@ public class ZeppelinConfiguration extends XMLConfiguration {
     return getRelativeDir(ConfVars.ZEPPELIN_K8S_TEMPLATE_DIR);
   }
 
+  public String getDockerContainerImage() {
+    return getString(ConfVars.ZEPPELIN_DOCKER_CONTAINER_IMAGE);
+  }
+
   public Map<String, String> dumpConfigurations(Predicate<String> predicate) {
     Map<String, String> properties = new HashMap<>();
 
@@ -887,7 +892,7 @@ public class ZeppelinConfiguration extends XMLConfiguration {
     ZEPPELIN_CLUSTER_HEARTBEAT_INTERVAL("zeppelin.cluster.heartbeat.interval", 3000),
     ZEPPELIN_CLUSTER_HEARTBEAT_TIMEOUT("zeppelin.cluster.heartbeat.timeout", 9000),
 
-    ZEPPELIN_RUN_MODE("zeppelin.run.mode", "auto"),              // auto | local | k8s
+    ZEPPELIN_RUN_MODE("zeppelin.run.mode", "auto"),              // auto | local | k8s | Docker
 
     ZEPPELIN_K8S_PORTFORWARD("zeppelin.k8s.portforward", false), // kubectl port-forward incase of Zeppelin is running outside of kuberentes
     ZEPPELIN_K8S_KUBECTL("zeppelin.k8s.kubectl", "kubectl"),     // kubectl command
@@ -895,6 +900,8 @@ public class ZeppelinConfiguration extends XMLConfiguration {
     ZEPPELIN_K8S_SPARK_CONTAINER_IMAGE("zeppelin.k8s.spark.container.image", "apache/spark:latest"),
     ZEPPELIN_K8S_TEMPLATE_DIR("zeppelin.k8s.template.dir", "k8s"),
 
+    ZEPPELIN_DOCKER_CONTAINER_IMAGE("zeppelin.docker.container.image", "apache/zeppelin:" + Util.getVersion()),
+
     ZEPPELIN_NOTEBOOK_GIT_REMOTE_URL("zeppelin.notebook.git.remote.url", ""),
     ZEPPELIN_NOTEBOOK_GIT_REMOTE_USERNAME("zeppelin.notebook.git.remote.username", "token"),
     ZEPPELIN_NOTEBOOK_GIT_REMOTE_ACCESS_TOKEN("zeppelin.notebook.git.remote.access-token", ""),
diff --git a/zeppelin-interpreter/src/main/java/org/apache/zeppelin/interpreter/launcher/InterpreterLauncher.java b/zeppelin-interpreter/src/main/java/org/apache/zeppelin/interpreter/launcher/InterpreterLauncher.java
index 30cf995..e505595 100644
--- a/zeppelin-interpreter/src/main/java/org/apache/zeppelin/interpreter/launcher/InterpreterLauncher.java
+++ b/zeppelin-interpreter/src/main/java/org/apache/zeppelin/interpreter/launcher/InterpreterLauncher.java
@@ -37,6 +37,10 @@ public abstract class InterpreterLauncher {
     this.recoveryStorage = recoveryStorage;
   }
 
+  public void setProperties(Properties props) {
+    this.properties = props;
+  }
+
   protected int getConnectTimeout() {
     int connectTimeout =
         zConf.getInt(ZeppelinConfiguration.ConfVars.ZEPPELIN_INTERPRETER_CONNECT_TIMEOUT);
diff --git a/zeppelin-plugins/launcher/docker/pom.xml b/zeppelin-plugins/launcher/docker/pom.xml
new file mode 100644
index 0000000..6d029fe
--- /dev/null
+++ b/zeppelin-plugins/launcher/docker/pom.xml
@@ -0,0 +1,116 @@
+<?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>zengine-plugins-parent</artifactId>
+    <groupId>org.apache.zeppelin</groupId>
+    <version>0.9.0-SNAPSHOT</version>
+    <relativePath>../../../zeppelin-plugins</relativePath>
+  </parent>
+
+  <groupId>org.apache.zeppelin</groupId>
+  <artifactId>launcher-docker</artifactId>
+  <packaging>jar</packaging>
+  <version>0.9.0-SNAPSHOT</version>
+  <name>Zeppelin: Plugin Docker Launcher</name>
+  <description>Launcher implementation to run interpreters on docker</description>
+
+  <properties>
+    <plugin.name>Launcher/DockerInterpreterLauncher</plugin.name>
+  </properties>
+
+  <dependencies>
+    <dependency>
+      <groupId>org.apache.zeppelin</groupId>
+      <artifactId>launcher-spark</artifactId>
+      <version>${project.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>com.hubspot.jinjava</groupId>
+      <artifactId>jinjava</artifactId>
+      <version>2.4.12</version>
+    </dependency>
+    <dependency>
+      <groupId>com.spotify</groupId>
+      <artifactId>docker-client</artifactId>
+      <classifier>shaded</classifier>
+      <version>8.15.2</version>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.commons</groupId>
+      <artifactId>commons-compress</artifactId>
+      <version>1.14</version>
+    </dependency>
+    <dependency>
+      <groupId>org.powermock</groupId>
+      <artifactId>powermock-api-mockito</artifactId>
+      <version>${powermock.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>org.powermock</groupId>
+      <artifactId>powermock-module-junit4</artifactId>
+      <version>${powermock.version}</version>
+    </dependency>
+  </dependencies>
+
+  <build>
+    <testResources>
+      <testResource>
+        <directory>${project.basedir}/src/test/resources</directory>
+      </testResource>
+      <testResource>
+        <directory>${project.basedir}/src/main/resources</directory>
+      </testResource>
+    </testResources>
+    <plugins>
+      <plugin>
+        <artifactId>maven-enforcer-plugin</artifactId>
+        <executions>
+          <execution>
+            <id>enforce</id>
+            <phase>none</phase>
+          </execution>
+        </executions>
+      </plugin>
+      <plugin>
+        <artifactId>maven-dependency-plugin</artifactId>
+      </plugin>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-checkstyle-plugin</artifactId>
+        <configuration>
+          <skip>false</skip>
+        </configuration>
+      </plugin>
+    </plugins>
+    <resources>
+      <resource>
+        <directory>src/main/resources</directory>
+        <includes>
+          <include>**/*.*</include>
+        </includes>
+      </resource>
+    </resources>
+  </build>
+</project>
diff --git a/zeppelin-plugins/launcher/docker/src/main/java/org/apache/zeppelin/interpreter/launcher/DockerInterpreterLauncher.java b/zeppelin-plugins/launcher/docker/src/main/java/org/apache/zeppelin/interpreter/launcher/DockerInterpreterLauncher.java
new file mode 100644
index 0000000..89a38d3
--- /dev/null
+++ b/zeppelin-plugins/launcher/docker/src/main/java/org/apache/zeppelin/interpreter/launcher/DockerInterpreterLauncher.java
@@ -0,0 +1,80 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES 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.interpreter.launcher;
+
+import org.apache.zeppelin.conf.ZeppelinConfiguration;
+import org.apache.zeppelin.interpreter.recovery.RecoveryStorage;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.Map;
+
+/**
+ * Interpreter Launcher which use shell script to launch the interpreter process.
+ */
+public class DockerInterpreterLauncher extends InterpreterLauncher {
+  private static final Logger LOGGER = LoggerFactory.getLogger(DockerInterpreterLauncher.class);
+
+  private InterpreterLaunchContext context;
+
+  public DockerInterpreterLauncher(ZeppelinConfiguration zConf, RecoveryStorage recoveryStorage)
+      throws IOException {
+    super(zConf, recoveryStorage);
+  }
+
+  @Override
+  public InterpreterClient launch(InterpreterLaunchContext context) throws IOException {
+    LOGGER.info("Launching Interpreter: " + context.getInterpreterSettingGroup());
+    this.context = context;
+    this.properties = context.getProperties();
+    int connectTimeout = getConnectTimeout();
+    if (connectTimeout < 200000) {
+      // DockerInterpreterLauncher needs to pull the image and create the container,
+      // it takes a long time, so the force is set to 200 seconds.
+      LOGGER.warn("DockerInterpreterLauncher needs to pull the image and create the container, " +
+          "it takes a long time, If the creation of the interpreter on docker fails, " +
+          "please increase the value of `zeppelin.interpreter.connect.timeout` " +
+          "in `zeppelin-site.xml`, recommend 200 seconds.");
+    }
+
+    StandardInterpreterLauncher interpreterLauncher = null;
+    if (isSpark()) {
+      interpreterLauncher = new SparkInterpreterLauncher(zConf, recoveryStorage);
+    } else {
+      interpreterLauncher = new StandardInterpreterLauncher(zConf, recoveryStorage);
+    }
+    interpreterLauncher.setProperties(context.getProperties());
+    Map<String, String> env = interpreterLauncher.buildEnvFromProperties(context);
+
+    return new DockerInterpreterProcess(
+        zConf,
+        zConf.getDockerContainerImage(),
+        context.getInterpreterGroupId(),
+        context.getInterpreterSettingGroup(),
+        context.getInterpreterSettingName(),
+        properties,
+        env,
+        context.getZeppelinServerHost(),
+        Integer.toString(context.getZeppelinServerRPCPort()),
+        connectTimeout);
+  }
+
+  boolean isSpark() {
+    return "spark".equalsIgnoreCase(context.getInterpreterSettingName());
+  }
+}
diff --git a/zeppelin-plugins/launcher/docker/src/main/java/org/apache/zeppelin/interpreter/launcher/DockerInterpreterProcess.java b/zeppelin-plugins/launcher/docker/src/main/java/org/apache/zeppelin/interpreter/launcher/DockerInterpreterProcess.java
new file mode 100644
index 0000000..b1374e7
--- /dev/null
+++ b/zeppelin-plugins/launcher/docker/src/main/java/org/apache/zeppelin/interpreter/launcher/DockerInterpreterProcess.java
@@ -0,0 +1,576 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES 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.interpreter.launcher;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Charsets;
+import com.google.common.collect.Maps;
+import com.google.common.io.Files;
+import com.google.common.io.Resources;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.net.SocketException;
+import java.net.URI;
+import java.net.URL;
+import java.net.UnknownHostException;
+import java.util.ArrayList;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import com.spotify.docker.client.DefaultDockerClient;
+import com.spotify.docker.client.DockerClient;
+import com.spotify.docker.client.LogStream;
+import com.spotify.docker.client.ProgressHandler;
+import com.spotify.docker.client.exceptions.DockerException;
+import com.spotify.docker.client.messages.ContainerConfig;
+import com.spotify.docker.client.messages.ContainerCreation;
+import com.spotify.docker.client.messages.ExecCreation;
+import com.spotify.docker.client.messages.HostConfig;
+import com.spotify.docker.client.messages.PortBinding;
+import com.spotify.docker.client.messages.ProgressMessage;
+import org.apache.commons.lang.StringUtils;
+import org.apache.zeppelin.conf.ZeppelinConfiguration;
+import org.apache.zeppelin.interpreter.launcher.utils.TarFileEntry;
+import org.apache.zeppelin.interpreter.launcher.utils.TarUtils;
+import org.apache.zeppelin.interpreter.remote.RemoteInterpreterProcess;
+import org.apache.zeppelin.interpreter.remote.RemoteInterpreterUtils;
+import org.apache.zeppelin.interpreter.thrift.RemoteInterpreterService;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.apache.zeppelin.conf.ZeppelinConfiguration.ConfVars.ZEPPELIN_SERVER_KERBEROS_KEYTAB;
+
+public class DockerInterpreterProcess extends RemoteInterpreterProcess {
+  private static final Logger LOGGER = LoggerFactory.getLogger(DockerInterpreterLauncher.class);
+
+  private String dockerIntpServicePort = "0";
+
+  private final String interpreterGroupId;
+  private final String interpreterGroupName;
+  private final String interpreterSettingName;
+  private final String containerImage;
+  private final Properties properties;
+  private final Map<String, String> envs;
+  private final String zeppelinServiceHost;
+  private final String zeppelinServiceRpcPort;
+
+  private AtomicBoolean dockerStarted = new AtomicBoolean(false);
+
+  private DockerClient docker = null;
+  private final String containerName;
+  private String containerHost = "";
+  private int containerPort = 0;
+  private static final String DOCKER_INTP_JINJA = "/jinja_templates/docker-interpreter.jinja";
+
+  // Upload local zeppelin library to container, There are several benefits
+  // Avoid the difference between the zeppelin version and the local in the container
+  // 1. RemoteInterpreterServer::main(String[] args), Different versions of args may be different
+  // 2. bin/interpreter.sh Start command args may be different
+  // 3. In the debugging phase for easy updating, Upload the local library file to container
+  @VisibleForTesting
+  boolean uploadLocalLibToContainter = true;
+
+  private ZeppelinConfiguration zconf;
+
+  private String zeppelinHome;
+
+  @VisibleForTesting
+  final String CONTAINER_SPARK_HOME;
+
+  @VisibleForTesting
+  final String DOCKER_HOST;
+
+  private String containerId;
+  final String CONTAINER_UPLOAD_TAR_DIR = "/tmp/zeppelin-tar";
+
+  public DockerInterpreterProcess(
+      ZeppelinConfiguration zconf,
+      String containerImage,
+      String interpreterGroupId,
+      String interpreterGroupName,
+      String interpreterSettingName,
+      Properties properties,
+      Map<String, String> envs,
+      String zeppelinServiceHost,
+      String zeppelinServiceRpcPort,
+      int connectTimeout
+  ) {
+    super(connectTimeout);
+
+    this.containerImage = containerImage;
+    this.interpreterGroupId = interpreterGroupId;
+    this.interpreterGroupName = interpreterGroupName;
+    this.interpreterSettingName = interpreterSettingName;
+    this.properties = properties;
+    this.envs = new HashMap(envs);
+    this.zeppelinServiceHost = zeppelinServiceHost;
+    this.zeppelinServiceRpcPort = zeppelinServiceRpcPort;
+
+    this.zconf = zconf;
+    this.containerName = interpreterGroupId.toLowerCase();
+
+    String sparkHome = System.getenv("CONTAINER_SPARK_HOME");
+    CONTAINER_SPARK_HOME = (sparkHome == null) ?  "/spark" : sparkHome;
+
+    String uploadLocalLib = System.getenv("UPLOAD_LOCAL_LIB_TO_CONTAINTER");
+    if (null != uploadLocalLib && StringUtils.equals(uploadLocalLib, "false")) {
+      uploadLocalLibToContainter = false;
+    }
+
+    try {
+      this.zeppelinHome = getZeppelinHome();
+    } catch (IOException e) {
+      LOGGER.error(e.getMessage(), e);
+    }
+    String defDockerHost = "http://0.0.0.0:2375";
+    String dockerHost = System.getenv("DOCKER_HOST");
+    DOCKER_HOST = (dockerHost == null) ?  defDockerHost : dockerHost;
+  }
+
+  @Override
+  public String getInterpreterSettingName() {
+    return interpreterSettingName;
+  }
+
+  @Override
+  public void start(String userName) throws IOException {
+    docker = DefaultDockerClient.builder().uri(URI.create(DOCKER_HOST)).build();
+
+    final Map<String, List<PortBinding>> portBindings = new HashMap<>();
+
+    // Bind container ports to host ports
+    int intpServicePort = RemoteInterpreterUtils.findRandomAvailablePortOnAllLocalInterfaces();
+    this.dockerIntpServicePort = String.valueOf(intpServicePort);
+    final String[] ports = {dockerIntpServicePort};
+    for (String port : ports) {
+      List<PortBinding> hostPorts = new ArrayList<>();
+      hostPorts.add(PortBinding.of("0.0.0.0", port));
+      portBindings.put(port, hostPorts);
+    }
+
+    final HostConfig hostConfig = HostConfig.builder()
+        .networkMode("host").portBindings(portBindings).build();
+
+    DockerSpecTemplate specTemplate = new DockerSpecTemplate();
+    specTemplate.loadProperties(getTemplateBindings());
+    URL urlTemplate = this.getClass().getResource(DOCKER_INTP_JINJA);
+    String template = Resources.toString(urlTemplate, Charsets.UTF_8);
+    String dockerCommand = specTemplate.render(template);
+    int firstLineIsNewline = dockerCommand.indexOf("\n");
+    if (firstLineIsNewline == 0) {
+      dockerCommand = dockerCommand.replaceFirst("\n", "");
+    }
+    LOGGER.info("dockerCommand = {}", dockerCommand);
+
+    List<String> listEnv = getListEnvs();
+    LOGGER.info("docker listEnv = {}", listEnv);
+
+    // Create container with exposed ports
+    final ContainerConfig containerConfig = ContainerConfig.builder()
+        .hostConfig(hostConfig)
+        .image(containerImage)
+        .workingDir("/")
+        .env(listEnv)
+        .cmd("sh", "-c", "while :; do sleep 1; done")
+        .build();
+
+    try {
+      LOGGER.info("wait docker pull image {} ...", containerImage);
+      docker.pull(containerImage, new ProgressHandler() {
+        @Override
+        public void progress(ProgressMessage message) throws DockerException {
+          if (null != message.error()) {
+            LOGGER.error(message.toString());
+          }
+        }
+      });
+
+      final ContainerCreation containerCreation
+          = docker.createContainer(containerConfig, containerName);
+      this.containerId = containerCreation.id();
+
+      // Start container
+      docker.startContainer(containerId);
+
+      copyRunFileToContainer(containerId);
+
+      execInContainer(containerId, dockerCommand, false);
+    } catch (DockerException e) {
+      LOGGER.error(e.getMessage(), e);
+    } catch (InterruptedException e) {
+      LOGGER.error(e.getMessage(), e);
+    }
+
+    long startTime = System.currentTimeMillis();
+
+    // wait until interpreter send dockerStarted message through thrift rpc
+    synchronized (dockerStarted) {
+      if (!dockerStarted.get()) {
+        try {
+          dockerStarted.wait(getConnectTimeout());
+        } catch (InterruptedException e) {
+          LOGGER.error("Remote interpreter is not accessible");
+        }
+      }
+    }
+
+    if (!dockerStarted.get()) {
+      LOGGER.info("Interpreter docker creation is time out in {} seconds",
+          getConnectTimeout() / 1000);
+    }
+
+    // waits for interpreter thrift rpc server ready
+    while (System.currentTimeMillis() - startTime < getConnectTimeout()) {
+      if (RemoteInterpreterUtils.checkIfRemoteEndpointAccessible(getHost(), getPort())) {
+        break;
+      } else {
+        try {
+          Thread.sleep(1000);
+        } catch (InterruptedException e) {
+          LOGGER.error(e.getMessage(), e);
+        }
+      }
+    }
+  }
+
+  @Override
+  public void processStarted(int port, String host) {
+    containerHost = host;
+    containerPort = port;
+    LOGGER.info("Interpreter container created {}:{}", containerHost, containerPort);
+    synchronized (dockerStarted) {
+      dockerStarted.set(true);
+      dockerStarted.notify();
+    }
+  }
+
+  @VisibleForTesting
+  Properties getTemplateBindings() throws IOException {
+    Properties dockerProperties = new Properties();
+
+    // docker template properties
+    dockerProperties.put("CONTAINER_ZEPPELIN_HOME", zeppelinHome);
+    dockerProperties.put("zeppelin.interpreter.container.image", containerImage);
+    dockerProperties.put("zeppelin.interpreter.group.id", interpreterGroupId);
+    dockerProperties.put("zeppelin.interpreter.group.name", interpreterGroupName);
+    dockerProperties.put("zeppelin.interpreter.setting.name", interpreterSettingName);
+    dockerProperties.put("zeppelin.interpreter.localRepo", "/tmp/local-repo");
+    dockerProperties.put("zeppelin.interpreter.rpc.portRange",
+        dockerIntpServicePort + ":" + dockerIntpServicePort);
+    dockerProperties.put("zeppelin.server.rpc.host", zeppelinServiceHost);
+    dockerProperties.put("zeppelin.server.rpc.portRange", zeppelinServiceRpcPort);
+
+    // interpreter properties overrides the values
+    dockerProperties.putAll(Maps.fromProperties(properties));
+
+    return dockerProperties;
+  }
+
+  @VisibleForTesting
+  List<String> getListEnvs() throws SocketException, UnknownHostException {
+    // environment variables
+    envs.put("ZEPPELIN_HOME", zeppelinHome);
+    envs.put("ZEPPELIN_CONF_DIR", zeppelinHome + "/conf");
+    envs.put("ZEPPELIN_FORCE_STOP", "true");
+    envs.put("SPARK_HOME", this.CONTAINER_SPARK_HOME);
+
+    List<String> listEnv = new ArrayList<>();
+    for (Map.Entry<String, String> entry : this.envs.entrySet()) {
+      String env = entry.getKey() + "=" + entry.getValue();
+      listEnv.add(env);
+    }
+    return listEnv;
+  }
+
+  @Override
+  public void stop() {
+    if (isRunning()) {
+      LOGGER.info("Kill interpreter process");
+      try {
+        callRemoteFunction(new RemoteFunction<Void>() {
+          @Override
+          public Void call(RemoteInterpreterService.Client client) throws Exception {
+            client.shutdown();
+            return null;
+          }
+        });
+      } catch (Exception e) {
+        LOGGER.warn("ignore the exception when shutting down", e);
+      }
+    }
+    try {
+      // Kill container
+      docker.killContainer(containerName);
+
+      // Remove container
+      docker.removeContainer(containerName);
+    } catch (DockerException e) {
+      e.printStackTrace();
+    } catch (InterruptedException e) {
+      e.printStackTrace();
+    }
+
+    // Close the docker client
+    docker.close();
+  }
+
+  @Override
+  public String getHost() {
+    return containerHost;
+  }
+
+  @Override
+  public int getPort() {
+    return containerPort;
+  }
+
+  @Override
+  public boolean isRunning() {
+    if (RemoteInterpreterUtils.checkIfRemoteEndpointAccessible(getHost(), getPort())) {
+      return true;
+    }
+    return false;
+  }
+
+  @Override
+  public String getErrorMessage() {
+    return null;
+  }
+
+  // upload configure file to submarine interpreter container
+  // keytab file & zeppelin-site.xml & krb5.conf
+  // The submarine configures the mount file into the container through `localization`
+  // NOTE: The path to the file uploaded to the container,
+  // Can not be repeated, otherwise it will lead to failure.
+  private void copyRunFileToContainer(String containerId)
+      throws IOException, DockerException, InterruptedException {
+    HashMap<String, String> copyFiles = new HashMap<>();
+
+    // Rebuild directory
+    rmInContainer(containerId, zeppelinHome);
+    mkdirInContainer(containerId, zeppelinHome);
+
+
+    // 1) zeppelin-site.xml is uploaded to `${CONTAINER_ZEPPELIN_HOME}` directory in the container
+    String confPath = "/conf";
+    String zeplConfPath = getPathByHome(zeppelinHome, confPath);
+    mkdirInContainer(containerId, zeplConfPath);
+    copyFiles.put(zeplConfPath + "/zeppelin-site.xml", zeplConfPath + "/zeppelin-site.xml");
+    copyFiles.put(zeplConfPath + "/log4j.properties", zeplConfPath + "/log4j.properties");
+    copyFiles.put(zeplConfPath + "/log4j_yarn_cluster.properties",
+        zeplConfPath + "/log4j_yarn_cluster.properties");
+
+    // 2) upload krb5.conf to container
+    String krb5conf = "/etc/krb5.conf";
+    File krb5File = new File(krb5conf);
+    if (krb5File.exists()) {
+      rmInContainer(containerId, krb5conf);
+      copyFiles.put(krb5conf, krb5conf);
+    } else {
+      LOGGER.warn("{} file not found, Did not upload the krb5.conf to the container!", krb5conf);
+    }
+
+    // 3) Get the keytab file in each interpreter properties
+    // Upload Keytab file to container, Keep the same directory as local host
+    // 3.1) shell interpreter properties keytab file
+    String intpKeytab = properties.getProperty("zeppelin.shell.keytab.location", "");
+    if (StringUtils.isBlank(intpKeytab)) {
+      // 3.2) spark interpreter properties keytab file
+      intpKeytab = properties.getProperty("spark.yarn.keytab", "");
+    }
+    if (StringUtils.isBlank(intpKeytab)) {
+      // 3.3) submarine interpreter properties keytab file
+      intpKeytab = properties.getProperty("submarine.hadoop.keytab", "");
+    }
+    if (StringUtils.isBlank(intpKeytab)) {
+      // 3.4) jdbc interpreter properties keytab file
+      intpKeytab = properties.getProperty("zeppelin.jdbc.keytab.location", "");
+    }
+    if (!StringUtils.isBlank(intpKeytab) && !copyFiles.containsKey(intpKeytab)) {
+      LOGGER.info("intpKeytab : {}", intpKeytab);
+      copyFiles.put(intpKeytab, intpKeytab);
+    }
+    // 3.5) zeppelin server keytab file
+    String zeppelinServerKeytab = zconf.getString(ZEPPELIN_SERVER_KERBEROS_KEYTAB);
+    if (!StringUtils.isBlank(zeppelinServerKeytab)
+        && !copyFiles.containsKey(zeppelinServerKeytab)) {
+      copyFiles.put(zeppelinServerKeytab, zeppelinServerKeytab);
+    }
+
+    // 4) hadoop conf dir
+    if (envs.containsKey("HADOOP_CONF_DIR")) {
+      String hadoopConfDir = envs.get("HADOOP_CONF_DIR");
+      copyFiles.put(hadoopConfDir, hadoopConfDir);
+    }
+
+    // 5) spark conf dir
+    if (envs.containsKey("SPARK_CONF_DIR")) {
+      String sparkConfDir = envs.get("SPARK_CONF_DIR");
+      rmInContainer(containerId, CONTAINER_SPARK_HOME + "/conf");
+      mkdirInContainer(containerId, CONTAINER_SPARK_HOME + "/conf");
+      copyFiles.put(sparkConfDir, CONTAINER_SPARK_HOME + "/conf");
+      envs.put("SPARK_CONF_DIR", CONTAINER_SPARK_HOME + "/conf");
+    }
+
+    if (uploadLocalLibToContainter){
+      // 6) ${ZEPPELIN_HOME}/bin is uploaded to `${CONTAINER_ZEPPELIN_HOME}`
+      //    directory in the container
+      String binPath = "/bin";
+      String zeplBinPath = getPathByHome(zeppelinHome, binPath);
+      mkdirInContainer(containerId, zeplBinPath);
+      docker.copyToContainer(new File(zeplBinPath).toPath(), containerId, zeplBinPath);
+
+      // 7) ${ZEPPELIN_HOME}/interpreter/spark is uploaded to `${CONTAINER_ZEPPELIN_HOME}`
+      //    directory in the container
+      String intpPath = "/interpreter/" + interpreterGroupName;
+      String zeplIntpPath = getPathByHome(zeppelinHome, intpPath);
+      mkdirInContainer(containerId, zeplIntpPath);
+      docker.copyToContainer(new File(zeplIntpPath).toPath(), containerId, zeplIntpPath);
+
+      // 8) ${ZEPPELIN_HOME}/lib/interpreter is uploaded to `${CONTAINER_ZEPPELIN_HOME}`
+      //    directory in the container
+      String libIntpPath = "/lib/interpreter";
+      String zeplLibIntpPath = getPathByHome(zeppelinHome, libIntpPath);
+      mkdirInContainer(containerId, zeplLibIntpPath);
+      docker.copyToContainer(new File(zeplLibIntpPath).toPath(), containerId, zeplLibIntpPath);
+    }
+
+    deployToContainer(containerId, copyFiles);
+  }
+
+  private void deployToContainer(String containerId, HashMap<String, String> copyFiles)
+      throws InterruptedException, DockerException, IOException {
+    // mkdir CONTAINER_UPLOAD_TAR_DIR
+    mkdirInContainer(containerId, CONTAINER_UPLOAD_TAR_DIR);
+
+    // file tar package
+    String tarFile = file2Tar(copyFiles);
+
+    // copy tar to ZEPPELIN_CONTAINER_DIR, auto unzip
+    InputStream inputStream = new FileInputStream(tarFile);
+    try {
+      docker.copyToContainer(inputStream, containerId, CONTAINER_UPLOAD_TAR_DIR);
+    } finally {
+      inputStream.close();
+    }
+
+    // copy all files in CONTAINER_UPLOAD_TAR_DIR to the root directory
+    cpdirInContainer(containerId, CONTAINER_UPLOAD_TAR_DIR + "/*", "/");
+
+    // delete tar file in the local
+    File fileTar = new File(tarFile);
+    fileTar.delete();
+  }
+
+  private void mkdirInContainer(String containerId, String path)
+      throws DockerException, InterruptedException {
+    String execCommand = "mkdir " + path + " -p";
+    execInContainer(containerId, execCommand, true);
+  }
+
+  private void rmInContainer(String containerId, String path)
+      throws DockerException, InterruptedException {
+    String execCommand = "rm " + path + " -R";
+    execInContainer(containerId, execCommand, true);
+  }
+
+  private void cpdirInContainer(String containerId, String from, String to)
+      throws DockerException, InterruptedException {
+    String execCommand = "cp " + from + " " + to + " -R";
+    execInContainer(containerId, execCommand, true);
+  }
+
+  private void execInContainer(String containerId, String execCommand, boolean logout)
+      throws DockerException, InterruptedException {
+
+    LOGGER.info("exec container commmand: " + execCommand);
+
+    final String[] command = {"sh", "-c", execCommand};
+    final ExecCreation execCreation = docker.execCreate(
+        containerId, command, DockerClient.ExecCreateParam.attachStdout(),
+        DockerClient.ExecCreateParam.attachStderr());
+
+    LogStream logStream = docker.execStart(execCreation.id());
+    while (logStream.hasNext() && logout) {
+      final String log = UTF_8.decode(logStream.next().content()).toString();
+      LOGGER.info(log);
+    }
+  }
+
+  private String file2Tar(HashMap<String, String> copyFiles) throws IOException {
+    File tmpDir = Files.createTempDir();
+
+    Date date = new Date();
+    String tarFileName = tmpDir.getPath() + date.getTime() + ".tar";
+
+    List<TarFileEntry> tarFileEntries = new ArrayList<>();
+    for (Map.Entry<String, String> entry : copyFiles.entrySet()) {
+      String filePath = entry.getKey();
+      String archivePath = entry.getValue();
+      TarFileEntry tarFileEntry = new TarFileEntry(new File(filePath), archivePath);
+      tarFileEntries.add(tarFileEntry);
+    }
+
+    TarUtils.compress(tarFileName, tarFileEntries);
+
+    return tarFileName;
+  }
+
+  @VisibleForTesting
+  boolean isSpark() {
+    return "spark".equalsIgnoreCase(interpreterGroupName);
+  }
+
+  private String getZeppelinHome() throws IOException {
+    String zeppelinHome = zconf.getZeppelinHome();
+    if (System.getenv("ZEPPELIN_HOME") != null) {
+      zeppelinHome = System.getenv("ZEPPELIN_HOME");
+    }
+
+    // check zeppelinHome is exist
+    File fileZeppelinHome = new File(zeppelinHome);
+    if (fileZeppelinHome.exists() && fileZeppelinHome.isDirectory()) {
+      return zeppelinHome;
+    }
+
+    throw new IOException("Can't find zeppelin home path!");
+  }
+
+  // ${ZEPPELIN_HOME}/interpreter/${interpreter-name}
+  // ${ZEPPELIN_HOME}/lib/interpreter
+  private String getPathByHome(String homeDir, String path) throws IOException {
+    File file = null;
+    if (null == homeDir || StringUtils.isEmpty(homeDir)) {
+      file = new File(path);
+    } else {
+      file = new File(homeDir, path);
+    }
+    if (file.exists()) {
+      return file.getAbsolutePath();
+    }
+
+    throw new IOException("Can't find directory in " + homeDir + path + "!");
+  }
+}
diff --git a/zeppelin-plugins/launcher/docker/src/main/java/org/apache/zeppelin/interpreter/launcher/DockerSpecTemplate.java b/zeppelin-plugins/launcher/docker/src/main/java/org/apache/zeppelin/interpreter/launcher/DockerSpecTemplate.java
new file mode 100644
index 0000000..c923f7e
--- /dev/null
+++ b/zeppelin-plugins/launcher/docker/src/main/java/org/apache/zeppelin/interpreter/launcher/DockerSpecTemplate.java
@@ -0,0 +1,78 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES 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.interpreter.launcher;
+
+import com.hubspot.jinjava.Jinjava;
+import org.apache.commons.io.FileUtils;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.charset.Charset;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Properties;
+import java.util.Set;
+
+public class DockerSpecTemplate extends HashMap<String, Object> {
+  public String render(File templateFile) throws IOException {
+    String template = FileUtils.readFileToString(templateFile, Charset.defaultCharset());
+    return render(template);
+  }
+
+  public String render(String template) {
+    ClassLoader oldClazzLoader = Thread.currentThread().getContextClassLoader();
+    try {
+      Thread.currentThread().setContextClassLoader(this.getClass().getClassLoader());
+      Jinjava jinja = new Jinjava();
+      return jinja.render(template, this);
+    } finally {
+      Thread.currentThread().setContextClassLoader(oldClazzLoader);
+    }
+  }
+
+  public void loadProperties(Properties properties) {
+    Set<Entry<Object, Object>> entries = properties.entrySet();
+    for (Entry entry : entries) {
+      String key = (String) entry.getKey();
+      Object value = entry.getValue();
+
+      String[] keySplit = key.split("[.]");
+      Map<String, Object> target = this;
+      for (int i = 0; i < keySplit.length - 1; i++) {
+        if (!target.containsKey(keySplit[i])) {
+          HashMap subEntry = new HashMap();
+          target.put(keySplit[i], subEntry);
+          target = subEntry;
+        } else {
+          Object subEntry = target.get(keySplit[i]);
+          if (!(subEntry instanceof Map)) {
+            HashMap replace = new HashMap();
+            replace.put("_", subEntry);
+            target.put(keySplit[i], replace);
+          }
+          target = (Map<String, Object>) target.get(keySplit[i]);
+        }
+      }
+
+      if (target.get(keySplit[keySplit.length - 1]) instanceof Map) {
+        ((Map) target.get(keySplit[keySplit.length - 1])).put("_", value);
+      } else {
+        target.put(keySplit[keySplit.length - 1], value);
+      }
+    }
+  }
+}
diff --git a/zeppelin-plugins/launcher/docker/src/main/java/org/apache/zeppelin/interpreter/launcher/utils/TarFileEntry.java b/zeppelin-plugins/launcher/docker/src/main/java/org/apache/zeppelin/interpreter/launcher/utils/TarFileEntry.java
new file mode 100644
index 0000000..04e28e7
--- /dev/null
+++ b/zeppelin-plugins/launcher/docker/src/main/java/org/apache/zeppelin/interpreter/launcher/utils/TarFileEntry.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.interpreter.launcher.utils;
+
+import java.io.File;
+
+public class TarFileEntry {
+  private File file;
+  private String archivePath;
+
+  public TarFileEntry(File file, String archivePath) {
+    this.file = file;
+    this.archivePath = archivePath;
+  }
+
+  public File getFile() {
+    return file;
+  }
+
+  public String getArchivePath() {
+    return archivePath;
+  }
+}
diff --git a/zeppelin-plugins/launcher/docker/src/main/java/org/apache/zeppelin/interpreter/launcher/utils/TarUtils.java b/zeppelin-plugins/launcher/docker/src/main/java/org/apache/zeppelin/interpreter/launcher/utils/TarUtils.java
new file mode 100644
index 0000000..343f3ee
--- /dev/null
+++ b/zeppelin-plugins/launcher/docker/src/main/java/org/apache/zeppelin/interpreter/launcher/utils/TarUtils.java
@@ -0,0 +1,103 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.zeppelin.interpreter.launcher.utils;
+
+import org.apache.commons.compress.archivers.tar.TarArchiveEntry;
+import org.apache.commons.compress.archivers.tar.TarArchiveInputStream;
+import org.apache.commons.compress.archivers.tar.TarArchiveOutputStream;
+import org.apache.commons.compress.compressors.gzip.GzipCompressorInputStream;
+import org.apache.commons.compress.compressors.gzip.GzipCompressorOutputStream;
+import org.apache.commons.compress.utils.IOUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.util.List;
+
+public class TarUtils {
+  private static final Logger LOGGER = LoggerFactory.getLogger(TarUtils.class);
+
+  public static void compress(String name, List<TarFileEntry> files) throws IOException {
+    try (TarArchiveOutputStream out = getTarArchiveOutputStream(name)){
+      for (TarFileEntry tarFileEntry : files){
+        addToArchiveCompression(out, tarFileEntry.getFile(), tarFileEntry.getArchivePath());
+      }
+    }
+  }
+
+  public static void decompress(String in, File out) throws IOException {
+    FileInputStream fileInputStream = new FileInputStream(in);
+    GzipCompressorInputStream gzipInputStream = new GzipCompressorInputStream(fileInputStream);
+
+    try (TarArchiveInputStream fin = new TarArchiveInputStream(gzipInputStream)){
+      TarArchiveEntry entry;
+      while ((entry = fin.getNextTarEntry()) != null) {
+        if (entry.isDirectory()) {
+          continue;
+        }
+        File curfile = new File(out, entry.getName());
+        File parent = curfile.getParentFile();
+        if (!parent.exists()) {
+          parent.mkdirs();
+        }
+        IOUtils.copy(fin, new FileOutputStream(curfile));
+      }
+    }
+  }
+
+  private static TarArchiveOutputStream getTarArchiveOutputStream(String name)
+      throws IOException {
+    FileOutputStream fileOutputStream = new FileOutputStream(name);
+    GzipCompressorOutputStream gzipOutputStream = new GzipCompressorOutputStream(fileOutputStream);
+    TarArchiveOutputStream taos = new TarArchiveOutputStream(gzipOutputStream);
+
+    // TAR has an 8 gig file limit by default, this gets around that
+    taos.setBigNumberMode(TarArchiveOutputStream.BIGNUMBER_STAR);
+
+    // TAR originally didn't support long file names, so enable the support for it
+    taos.setLongFileMode(TarArchiveOutputStream.LONGFILE_GNU);
+    taos.setAddPaxHeadersForNonAsciiNames(true);
+
+    return taos;
+  }
+
+  private static void addToArchiveCompression(TarArchiveOutputStream out, File file, String dir)
+      throws IOException {
+    if (file.isFile()){
+      String archivePath = "." + dir;
+      LOGGER.info("archivePath = " + archivePath);
+      out.putArchiveEntry(new TarArchiveEntry(file, archivePath));
+      try (FileInputStream in = new FileInputStream(file)) {
+        IOUtils.copy(in, out);
+      }
+      out.closeArchiveEntry();
+    } else if (file.isDirectory()) {
+      File[] children = file.listFiles();
+      if (children != null){
+        for (File child : children){
+          String appendDir = child.getAbsolutePath().replace(file.getAbsolutePath(), "");
+          addToArchiveCompression(out, child, dir + appendDir);
+        }
+      }
+    } else {
+      LOGGER.error(file.getName() + " is not supported");
+    }
+  }
+}
diff --git a/zeppelin-plugins/launcher/docker/src/main/resources/jinja_templates/docker-interpreter.jinja b/zeppelin-plugins/launcher/docker/src/main/resources/jinja_templates/docker-interpreter.jinja
new file mode 100644
index 0000000..98c42a5
--- /dev/null
+++ b/zeppelin-plugins/launcher/docker/src/main/resources/jinja_templates/docker-interpreter.jinja
@@ -0,0 +1,17 @@
+{#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#}
+{{CONTAINER_ZEPPELIN_HOME}}/bin/interpreter.sh -d {{CONTAINER_ZEPPELIN_HOME}}/interpreter/{{zeppelin.interpreter.group.name}} -r {{zeppelin.interpreter.rpc.portRange}} -c {{zeppelin.server.rpc.host}} -p {{zeppelin.server.rpc.portRange}} -i {{zeppelin.interpreter.group.id}} -l {{zeppelin.interpreter.localRepo}} -g {{zeppelin.interpreter.setting.name}}
\ No newline at end of file
diff --git a/zeppelin-plugins/launcher/docker/src/test/java/org/apache/zeppelin/interpreter/launcher/DockerInterpreterProcessTest.java b/zeppelin-plugins/launcher/docker/src/test/java/org/apache/zeppelin/interpreter/launcher/DockerInterpreterProcessTest.java
new file mode 100644
index 0000000..ce35b79
--- /dev/null
+++ b/zeppelin-plugins/launcher/docker/src/test/java/org/apache/zeppelin/interpreter/launcher/DockerInterpreterProcessTest.java
@@ -0,0 +1,149 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES 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.interpreter.launcher;
+
+import org.apache.zeppelin.conf.ZeppelinConfiguration;
+import org.apache.zeppelin.interpreter.InterpreterOption;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.powermock.api.mockito.PowerMockito;
+import org.powermock.core.classloader.annotations.PrepareForTest;
+import org.powermock.modules.junit4.PowerMockRunner;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotEquals;
+import static org.junit.Assert.assertTrue;
+
+@RunWith(PowerMockRunner.class)
+@PrepareForTest({System.class, DockerInterpreterProcess.class})
+public class DockerInterpreterProcessTest {
+  private static final Logger LOGGER = LoggerFactory.getLogger(DockerInterpreterProcessTest.class);
+
+  protected static ZeppelinConfiguration zconf = ZeppelinConfiguration.create();
+
+  @Test
+  public void testCreateIntpProcess() throws IOException {
+    DockerInterpreterLauncher launcher
+        = new DockerInterpreterLauncher(zconf, null);
+    Properties properties = new Properties();
+    properties.setProperty(
+        ZeppelinConfiguration.ConfVars.ZEPPELIN_INTERPRETER_CONNECT_TIMEOUT.getVarName(), "5000");
+    InterpreterOption option = new InterpreterOption();
+    InterpreterLaunchContext context = new InterpreterLaunchContext(properties, option, null,
+        "user1", "intpGroupId", "groupId",
+        "groupName", "name", 0, "host");
+    InterpreterClient client = launcher.launch(context);
+
+    assertTrue(client instanceof DockerInterpreterProcess);
+    DockerInterpreterProcess interpreterProcess = (DockerInterpreterProcess) client;
+    assertEquals("name", interpreterProcess.getInterpreterSettingName());
+
+    assertEquals(interpreterProcess.CONTAINER_SPARK_HOME, "/spark");
+    assertEquals(interpreterProcess.uploadLocalLibToContainter, true);
+    assertNotEquals(interpreterProcess.DOCKER_HOST, "http://my-docker-host:2375");
+  }
+
+  @Test
+  public void testEnv() throws IOException {
+    PowerMockito.mockStatic(System.class);
+    PowerMockito.when(System.getenv("CONTAINER_SPARK_HOME")).thenReturn("my-spark-home");
+    PowerMockito.when(System.getenv("UPLOAD_LOCAL_LIB_TO_CONTAINTER")).thenReturn("false");
+    PowerMockito.when(System.getenv("DOCKER_HOST")).thenReturn("http://my-docker-host:2375");
+
+    Properties properties = new Properties();
+    properties.setProperty(
+        ZeppelinConfiguration.ConfVars.ZEPPELIN_INTERPRETER_CONNECT_TIMEOUT.getVarName(), "5000");
+
+    HashMap<String, String> envs = new HashMap<String, String>();
+    envs.put("MY_ENV1", "V1");
+
+    DockerInterpreterProcess intp = new DockerInterpreterProcess(
+        zconf,
+        "interpreter-container:1.0",
+        "shared_process",
+        "sh",
+        "shell",
+        properties,
+        envs,
+        "zeppelin.server.hostname",
+        "12320",
+        5000);
+
+    assertEquals(intp.CONTAINER_SPARK_HOME, "my-spark-home");
+    assertEquals(intp.uploadLocalLibToContainter, false);
+    assertEquals(intp.DOCKER_HOST, "http://my-docker-host:2375");
+  }
+
+  @Test
+  public void testTemplateBindings() throws IOException {
+    Properties properties = new Properties();
+    properties.setProperty(
+        ZeppelinConfiguration.ConfVars.ZEPPELIN_INTERPRETER_CONNECT_TIMEOUT.getVarName(), "5000");
+
+    HashMap<String, String> envs = new HashMap<String, String>();
+    envs.put("MY_ENV1", "V1");
+
+    DockerInterpreterProcess intp = new DockerInterpreterProcess(
+        zconf,
+        "interpreter-container:1.0",
+        "shared_process",
+        "sh",
+        "shell",
+        properties,
+        envs,
+        "zeppelin.server.hostname",
+        "12320",
+        5000);
+
+    Properties dockerProperties = intp.getTemplateBindings();
+    assertEquals(dockerProperties.size(), 10);
+
+    assertTrue(null != dockerProperties.get("CONTAINER_ZEPPELIN_HOME"));
+    assertTrue(null != dockerProperties.get("zeppelin.interpreter.container.image"));
+    assertTrue(null != dockerProperties.get("zeppelin.interpreter.group.id"));
+    assertTrue(null != dockerProperties.get("zeppelin.interpreter.group.name"));
+    assertTrue(null != dockerProperties.get("zeppelin.interpreter.setting.name"));
+    assertTrue(null != dockerProperties.get("zeppelin.interpreter.localRepo"));
+    assertTrue(null != dockerProperties.get("zeppelin.interpreter.rpc.portRange"));
+    assertTrue(null != dockerProperties.get("zeppelin.server.rpc.host"));
+    assertTrue(null != dockerProperties.get("zeppelin.server.rpc.portRange"));
+    assertTrue(null != dockerProperties.get("zeppelin.interpreter.connect.timeout"));
+
+    List<String> listEnvs = intp.getListEnvs();
+    assertEquals(listEnvs.size(), 5);
+    Map<String, String> mapEnv = new HashMap<>();
+    for (int i = 0; i < listEnvs.size(); i++) {
+      String env = listEnvs.get(i);
+      String kv[] = env.split("=");
+      mapEnv.put(kv[0], kv[1]);
+    }
+    assertEquals(mapEnv.size(), 5);
+    assertTrue(mapEnv.containsKey("ZEPPELIN_HOME"));
+    assertTrue(mapEnv.containsKey("ZEPPELIN_CONF_DIR"));
+    assertTrue(mapEnv.containsKey("ZEPPELIN_FORCE_STOP"));
+    assertTrue(mapEnv.containsKey("SPARK_HOME"));
+    assertTrue(mapEnv.containsKey("MY_ENV1"));
+  }
+}
diff --git a/zeppelin-plugins/pom.xml b/zeppelin-plugins/pom.xml
index 727acd7..a9fad2e 100644
--- a/zeppelin-plugins/pom.xml
+++ b/zeppelin-plugins/pom.xml
@@ -48,6 +48,7 @@
         <module>launcher/k8s-standard</module>
         <module>launcher/spark</module>
         <module>launcher/cluster</module>
+        <module>launcher/docker</module>
     </modules>
 
     <dependencies>
diff --git a/zeppelin-zengine/src/main/java/org/apache/zeppelin/interpreter/InterpreterSetting.java b/zeppelin-zengine/src/main/java/org/apache/zeppelin/interpreter/InterpreterSetting.java
index 2fbef23..fb15e74 100644
--- a/zeppelin-zengine/src/main/java/org/apache/zeppelin/interpreter/InterpreterSetting.java
+++ b/zeppelin-zengine/src/main/java/org/apache/zeppelin/interpreter/InterpreterSetting.java
@@ -672,6 +672,8 @@ public class InterpreterSetting {
       return "K8sStandardInterpreterLauncher";
     } else if (isRunningOnCluster()) {
       return "ClusterInterpreterLauncher";
+    } if (isRunningOnDocker()) {
+      return "DockerInterpreterLauncher";
     } else {
       if (group.equals("spark")) {
         return "SparkInterpreterLauncher";
@@ -685,10 +687,15 @@ public class InterpreterSetting {
     return conf.getRunMode() == ZeppelinConfiguration.RUN_MODE.K8S;
   }
 
+
   private boolean isRunningOnCluster() {
     return conf.isClusterMode();
   }
 
+  private boolean isRunningOnDocker() {
+    return conf.getRunMode() == ZeppelinConfiguration.RUN_MODE.DOCKER;
+  }
+
   public boolean isUserAuthorized(List<String> userAndRoles) {
     if (!option.permissionIsSet()) {
       return true;