You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@zeppelin.apache.org by jo...@apache.org on 2018/04/24 02:55:08 UTC

zeppelin git commit: ZEPPELIN-3382 Installing interpreter from UI

Repository: zeppelin
Updated Branches:
  refs/heads/master 75dd1958e -> 5fdbd5338


ZEPPELIN-3382 Installing interpreter from UI

### What is this PR for?
Installing a new interpreter from helium's UI. You just can click enable button to download a new interpreter you want to install

### What type of PR is it?
[Improvement]

### Todos
* [x] - Changed frontend side to enable users to click an install button
* [x] - Added status message for installation
* [x] - Added Backend logic to handle an interpreter installation

### What is the Jira issue?
* https://issues.apache.org/jira/browse/ZEPPELIN-3382

### How should this be tested?
1. Make sure you don't have `solr` interpreter in an interpreter create menu
1. Move to Helium/Interpreter menu and click the `solr` enable button
1. See the messages for downloading it and having downloaded it
1. Check interpreter creation menu to make sure `solr` exists

### Screenshots (if appropriate)
![image](https://user-images.githubusercontent.com/3612566/38859396-e3548986-4268-11e8-9516-aeb1e87c85b3.png)
![image](https://user-images.githubusercontent.com/3612566/38859408-e9e66274-4268-11e8-8445-54d13b5c2a08.png)
![image](https://user-images.githubusercontent.com/3612566/38859588-50238418-4269-11e8-93d8-df8a125b0fcb.png)

### Questions:
* Does the licenses files need update? No
* Is there breaking changes for older versions? No
* Does this needs documentation? Might be

Author: Jongyoul Lee <jo...@gmail.com>

Closes #2905 from jongyoul/ZEPPELIN-3382 and squashes the following commits:

e66dd2a50 [Jongyoul Lee] Changed a parameter name Fixed test to check callback's messages
ce7f471bd [Jongyoul Lee] Changed the name of a callback class from `MessageCallback` to `ServiceCallback`
82f24e320 [Jongyoul Lee] Added the logic to handle "zeppelin-*" case
e6ce00f4e [Jongyoul Lee] Refactored callback to hide callback-specific information
483b74c82 [Jongyoul Lee] Removed the default constructor of InterpreterRestApi Changed MessageCallback to have onStart, onSuccess, onFailure Added downloaded folder info
10d5a9faa [Jongyoul Lee] Removed spec-out configuration, test settings, and file of ZEPPELIN_INTERPRETER_DIR_REFRESH_INTERVAL Removed unused import statements
4112dd2da [Jongyoul Lee] Removed spec-out test
3b801a77f [Jongyoul Lee] Fixed typo
48217749a [Jongyoul Lee] Fixed tests
8f0c94aa5 [Jongyoul Lee] Introduced MessageCallback to handle message from InterpreterService to frontend
3ddf8deb9 [Jongyoul Lee] Introduce InterpreterInstallationRequest.fromJson method
ee4e89500 [Jongyoul Lee] Fixed styles
d7d88eec2 [Jongyoul Lee] Fixed js lint
2adf40416 [Jongyoul Lee] Replaced ThreadFactoryFactory to ThreadFactoryBuilder
25bc09faa [Jongyoul Lee] Reverted package-lock.json
ab668bb87 [Jongyoul Lee] Added a license header
735f76346 [Jongyoul Lee] Removed a scheduler for refreshing interpreters from a directory
0a0189b04 [Jongyoul Lee] Added new message to send the status to client Added frontend logic to install interpreter.
ff6f5f9a4 [Jongyoul Lee] Added new api for installInterpreter Added InterpreterService class to handle of installing interpreter
91768f66e [Jongyoul Lee] Implemented backend side
f6a2eb0a1 [Jongyoul Lee] Added scheduler to check the interpreter directory Added test case for refreshing list


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

Branch: refs/heads/master
Commit: 5fdbd5338293dc32f238c834036a7af409d25986
Parents: 75dd195
Author: Jongyoul Lee <jo...@gmail.com>
Authored: Mon Apr 23 16:15:21 2018 +0900
Committer: Jongyoul Lee <jo...@apache.org>
Committed: Tue Apr 24 11:55:03 2018 +0900

----------------------------------------------------------------------
 .../zeppelin/conf/ZeppelinConfiguration.java    |  17 +-
 .../zeppelin/rest/InterpreterRestApi.java       |  75 ++++++++-
 .../message/InterpreterInstallationRequest.java |  44 +++++
 .../apache/zeppelin/server/ZeppelinServer.java  |   9 +-
 .../zeppelin/service/InterpreterService.java    | 159 +++++++++++++++++++
 .../apache/zeppelin/socket/NotebookServer.java  |  12 ++
 .../apache/zeppelin/socket/ServiceCallback.java |  27 ++++
 .../service/InterpreterServiceTest.java         | 142 +++++++++++++++++
 .../src/app/helium/helium.controller.js         |  40 +++--
 zeppelin-web/src/app/helium/helium.service.js   |   4 +
 .../websocket/websocket-event.factory.js        |   6 +-
 .../interpreter/InterpreterSettingManager.java  |  96 ++++++-----
 .../zeppelin/notebook/socket/Message.java       |   4 +-
 .../interpreter/AbstractInterpreterTest.java    |   1 -
 .../InterpreterSettingManagerTest.java          |   1 -
 15 files changed, 569 insertions(+), 68 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/zeppelin/blob/5fdbd533/zeppelin-interpreter/src/main/java/org/apache/zeppelin/conf/ZeppelinConfiguration.java
----------------------------------------------------------------------
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 061e230..1aedb7f 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
@@ -607,6 +607,18 @@ public class ZeppelinConfiguration extends XMLConfiguration {
     return getString(ConfVars.ZEPPELIN_NOTEBOOK_CRON_FOLDERS);
   }
 
+  public String getZeppelinProxyUrl() {
+    return getString(ConfVars.ZEPPELIN_PROXY_URL);
+  }
+
+  public String getZeppelinProxyUser() {
+    return getString(ConfVars.ZEPPELIN_PROXY_USER);
+  }
+
+  public String getZeppelinProxyPassword() {
+    return getString(ConfVars.ZEPPELIN_PROXY_PASSWORD);
+  }
+
   public Map<String, String> dumpConfigurations(ZeppelinConfiguration conf,
                                                 ConfigurationKeyPredicate predicate) {
     Map<String, String> configurations = new HashMap<>();
@@ -802,7 +814,10 @@ public class ZeppelinConfiguration extends XMLConfiguration {
     ZEPPELIN_NOTEBOOK_GIT_REMOTE_ACCESS_TOKEN("zeppelin.notebook.git.remote.access-token", ""),
     ZEPPELIN_NOTEBOOK_GIT_REMOTE_ORIGIN("zeppelin.notebook.git.remote.origin", "origin"),
     ZEPPELIN_NOTEBOOK_CRON_ENABLE("zeppelin.notebook.cron.enable", false),
-    ZEPPELIN_NOTEBOOK_CRON_FOLDERS("zeppelin.notebook.cron.folders", null);
+    ZEPPELIN_NOTEBOOK_CRON_FOLDERS("zeppelin.notebook.cron.folders", null),
+    ZEPPELIN_PROXY_URL("zeppelin.proxy.url", null),
+    ZEPPELIN_PROXY_USER("zeppelin.proxy.user", null),
+    ZEPPELIN_PROXY_PASSWORD("zeppelin.proxy.password", null);
 
     private String varName;
     @SuppressWarnings("rawtypes")

http://git-wip-us.apache.org/repos/asf/zeppelin/blob/5fdbd533/zeppelin-server/src/main/java/org/apache/zeppelin/rest/InterpreterRestApi.java
----------------------------------------------------------------------
diff --git a/zeppelin-server/src/main/java/org/apache/zeppelin/rest/InterpreterRestApi.java b/zeppelin-server/src/main/java/org/apache/zeppelin/rest/InterpreterRestApi.java
index 479740d..2960f37 100644
--- a/zeppelin-server/src/main/java/org/apache/zeppelin/rest/InterpreterRestApi.java
+++ b/zeppelin-server/src/main/java/org/apache/zeppelin/rest/InterpreterRestApi.java
@@ -14,9 +14,15 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+
 package org.apache.zeppelin.rest;
 
+import com.google.common.collect.Maps;
+import javax.validation.constraints.NotNull;
 import org.apache.commons.lang.exception.ExceptionUtils;
+import org.apache.zeppelin.notebook.socket.Message;
+import org.apache.zeppelin.notebook.socket.Message.OP;
+import org.apache.zeppelin.socket.ServiceCallback;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.sonatype.aether.repository.RemoteRepository;
@@ -43,10 +49,12 @@ import org.apache.zeppelin.interpreter.InterpreterException;
 import org.apache.zeppelin.interpreter.InterpreterPropertyType;
 import org.apache.zeppelin.interpreter.InterpreterSetting;
 import org.apache.zeppelin.interpreter.InterpreterSettingManager;
+import org.apache.zeppelin.rest.message.InterpreterInstallationRequest;
 import org.apache.zeppelin.rest.message.NewInterpreterSettingRequest;
 import org.apache.zeppelin.rest.message.RestartInterpreterRequest;
 import org.apache.zeppelin.rest.message.UpdateInterpreterSettingRequest;
 import org.apache.zeppelin.server.JsonResponse;
+import org.apache.zeppelin.service.InterpreterService;
 import org.apache.zeppelin.socket.NotebookServer;
 import org.apache.zeppelin.utils.SecurityUtils;
 
@@ -56,16 +64,18 @@ import org.apache.zeppelin.utils.SecurityUtils;
 @Path("/interpreter")
 @Produces("application/json")
 public class InterpreterRestApi {
-  private static final Logger logger = LoggerFactory.getLogger(InterpreterRestApi.class);
 
-  private InterpreterSettingManager interpreterSettingManager;
-  private NotebookServer notebookServer;
+  private static final Logger logger = LoggerFactory.getLogger(InterpreterRestApi.class);
 
-  public InterpreterRestApi() {
-  }
+  private final InterpreterService interpreterService;
+  private final InterpreterSettingManager interpreterSettingManager;
+  private final NotebookServer notebookServer;
 
-  public InterpreterRestApi(InterpreterSettingManager interpreterSettingManager,
-          NotebookServer notebookWsServer) {
+  public InterpreterRestApi(
+      InterpreterService interpreterService,
+      InterpreterSettingManager interpreterSettingManager,
+      NotebookServer notebookWsServer) {
+    this.interpreterService = interpreterService;
     this.interpreterSettingManager = interpreterSettingManager;
     this.notebookServer = notebookWsServer;
   }
@@ -281,11 +291,60 @@ public class InterpreterRestApi {
   }
 
   /**
-   * Get available types for property.
+   * Get available types for property
    */
   @GET
   @Path("property/types")
   public Response listInterpreterPropertyTypes() {
     return new JsonResponse<>(Status.OK, InterpreterPropertyType.getTypes()).build();
   }
+
+  /** Install interpreter */
+  @POST
+  @Path("install")
+  @ZeppelinApi
+  public Response installInterpreter(@NotNull String message) {
+    logger.info("Install interpreter: {}", message);
+    InterpreterInstallationRequest request = InterpreterInstallationRequest.fromJson(message);
+
+    try {
+      interpreterService.installInterpreter(
+          request,
+          new ServiceCallback() {
+            @Override
+            public void onStart(String message) {
+              Message m = new Message(OP.INTERPRETER_INSTALL_STARTED);
+              Map<String, Object> data = Maps.newHashMap();
+              data.put("result", "Starting");
+              data.put("message", message);
+              m.data = data;
+              notebookServer.broadcast(m);
+            }
+
+            @Override
+            public void onSuccess(String message) {
+              Message m = new Message(OP.INTERPRETER_INSTALL_RESULT);
+              Map<String, Object> data = Maps.newHashMap();
+              data.put("result", "Succeed");
+              data.put("message", message);
+              m.data = data;
+              notebookServer.broadcast(m);
+            }
+
+            @Override
+            public void onFailure(String message) {
+              Message m = new Message(OP.INTERPRETER_INSTALL_RESULT);
+              Map<String, Object> data = Maps.newHashMap();
+              data.put("result", "Failed");
+              data.put("message", message);
+              m.data = data;
+              notebookServer.broadcast(m);
+            }
+          });
+    } catch (Throwable t) {
+      return new JsonResponse<>(Status.INTERNAL_SERVER_ERROR, t.getMessage()).build();
+    }
+
+    return new JsonResponse<>(Status.OK).build();
+  }
 }

http://git-wip-us.apache.org/repos/asf/zeppelin/blob/5fdbd533/zeppelin-server/src/main/java/org/apache/zeppelin/rest/message/InterpreterInstallationRequest.java
----------------------------------------------------------------------
diff --git a/zeppelin-server/src/main/java/org/apache/zeppelin/rest/message/InterpreterInstallationRequest.java b/zeppelin-server/src/main/java/org/apache/zeppelin/rest/message/InterpreterInstallationRequest.java
new file mode 100644
index 0000000..9f92794
--- /dev/null
+++ b/zeppelin-server/src/main/java/org/apache/zeppelin/rest/message/InterpreterInstallationRequest.java
@@ -0,0 +1,44 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES 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.rest.message;
+
+import com.google.gson.Gson;
+import com.google.gson.GsonBuilder;
+
+public class InterpreterInstallationRequest {
+  private static final Gson gson = new GsonBuilder().create();
+  private String name;
+  private String artifact;
+
+  public InterpreterInstallationRequest(String name, String artifact) {
+    this.name = name;
+    this.artifact = artifact;
+  }
+
+  public static InterpreterInstallationRequest fromJson(String message) {
+    return gson.fromJson(message, InterpreterInstallationRequest.class);
+  }
+
+  public String getName() {
+    return name;
+  }
+
+  public String getArtifact() {
+    return artifact;
+  }
+}

http://git-wip-us.apache.org/repos/asf/zeppelin/blob/5fdbd533/zeppelin-server/src/main/java/org/apache/zeppelin/server/ZeppelinServer.java
----------------------------------------------------------------------
diff --git a/zeppelin-server/src/main/java/org/apache/zeppelin/server/ZeppelinServer.java b/zeppelin-server/src/main/java/org/apache/zeppelin/server/ZeppelinServer.java
index 0f4eedf..a6b9813 100644
--- a/zeppelin-server/src/main/java/org/apache/zeppelin/server/ZeppelinServer.java
+++ b/zeppelin-server/src/main/java/org/apache/zeppelin/server/ZeppelinServer.java
@@ -69,6 +69,7 @@ import org.apache.zeppelin.rest.ZeppelinRestApi;
 import org.apache.zeppelin.scheduler.SchedulerFactory;
 import org.apache.zeppelin.search.LuceneSearch;
 import org.apache.zeppelin.search.SearchService;
+import org.apache.zeppelin.service.InterpreterService;
 import org.apache.zeppelin.socket.NotebookServer;
 import org.apache.zeppelin.storage.ConfigStorage;
 import org.apache.zeppelin.user.Credentials;
@@ -93,6 +94,7 @@ public class ZeppelinServer extends Application {
   private NotebookRepoSync notebookRepo;
   private NotebookAuthorization notebookAuthorization;
   private Credentials credentials;
+  private InterpreterService interpreterService;
 
   public ZeppelinServer() throws Exception {
     ZeppelinConfiguration conf = ZeppelinConfiguration.create();
@@ -156,7 +158,7 @@ public class ZeppelinServer extends Application {
     } catch (Exception e) {
       LOG.error(e.getMessage(), e);
     }
-    
+
     // to update notebook from application event from remote process.
     heliumApplicationFactory.setNotebook(notebook);
     // to update fire websocket event on application event.
@@ -164,6 +166,7 @@ public class ZeppelinServer extends Application {
 
     notebook.addNotebookEventListener(heliumApplicationFactory);
     notebook.addNotebookEventListener(notebookWsServer.getNotebookInformationListener());
+    this.interpreterService = new InterpreterService(conf, interpreterSettingManager);
   }
 
   public static void main(String[] args) throws InterruptedException {
@@ -395,8 +398,8 @@ public class ZeppelinServer extends Application {
     HeliumRestApi heliumApi = new HeliumRestApi(helium, notebook);
     singletons.add(heliumApi);
 
-    InterpreterRestApi interpreterApi = new InterpreterRestApi(interpreterSettingManager,
-        notebookWsServer);
+    InterpreterRestApi interpreterApi = new InterpreterRestApi(interpreterService,
+        interpreterSettingManager, notebookWsServer);
     singletons.add(interpreterApi);
 
     CredentialRestApi credentialApi = new CredentialRestApi(credentials);

http://git-wip-us.apache.org/repos/asf/zeppelin/blob/5fdbd533/zeppelin-server/src/main/java/org/apache/zeppelin/service/InterpreterService.java
----------------------------------------------------------------------
diff --git a/zeppelin-server/src/main/java/org/apache/zeppelin/service/InterpreterService.java b/zeppelin-server/src/main/java/org/apache/zeppelin/service/InterpreterService.java
new file mode 100644
index 0000000..a66bc71
--- /dev/null
+++ b/zeppelin-server/src/main/java/org/apache/zeppelin/service/InterpreterService.java
@@ -0,0 +1,159 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES 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.service;
+
+import com.google.common.collect.Lists;
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+import java.io.IOException;
+import java.net.MalformedURLException;
+import java.net.URL;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.List;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import jline.internal.Preconditions;
+import org.apache.commons.io.FileUtils;
+import org.apache.zeppelin.conf.ZeppelinConfiguration;
+import org.apache.zeppelin.dep.DependencyResolver;
+import org.apache.zeppelin.interpreter.InterpreterSettingManager;
+import org.apache.zeppelin.rest.message.InterpreterInstallationRequest;
+import org.apache.zeppelin.socket.ServiceCallback;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.sonatype.aether.RepositoryException;
+
+/**
+ * This class handles all of business logic for {@link org.apache.zeppelin.rest.InterpreterRestApi}
+ */
+public class InterpreterService {
+
+  private static final String ZEPPELIN_ARTIFACT_PREFIX = "zeppelin-";
+  private static final Logger logger = LoggerFactory.getLogger(InterpreterService.class);
+  private static final ExecutorService executorService =
+      Executors.newSingleThreadExecutor(
+          new ThreadFactoryBuilder()
+              .setNameFormat(InterpreterService.class.getSimpleName() + "-")
+              .build());
+
+  private final ZeppelinConfiguration conf;
+  private final InterpreterSettingManager interpreterSettingManager;
+
+  public InterpreterService(
+      ZeppelinConfiguration conf, InterpreterSettingManager interpreterSettingManager) {
+    this.conf = conf;
+    this.interpreterSettingManager = interpreterSettingManager;
+  }
+
+  public void installInterpreter(
+      final InterpreterInstallationRequest request, final ServiceCallback serviceCallback)
+      throws Exception {
+    Preconditions.checkNotNull(request);
+    String interpreterName = request.getName();
+    Preconditions.checkNotNull(interpreterName);
+    Preconditions.checkNotNull(request.getArtifact());
+
+    String interpreterBaseDir = conf.getInterpreterDir();
+    String localRepoPath = conf.getInterpreterLocalRepoPath();
+
+    final DependencyResolver dependencyResolver = new DependencyResolver(localRepoPath);
+
+    String proxyUrl = conf.getZeppelinProxyUrl();
+    if (null != proxyUrl) {
+      String proxyUser = conf.getZeppelinProxyUser();
+      String proxyPassword = conf.getZeppelinProxyPassword();
+      try {
+        dependencyResolver.setProxy(new URL(proxyUrl), proxyUser, proxyPassword);
+      } catch (MalformedURLException e) {
+        // TODO(jl): Not sure if it's good to raise an exception
+        throw new Exception("Url is not valid format", e);
+      }
+    }
+
+    // TODO(jl): Make a rule between an interpreter name and an installation directory
+    List<String> possibleInterpreterDirectories = Lists.newArrayList();
+    possibleInterpreterDirectories.add(interpreterName);
+    if (interpreterName.startsWith(ZEPPELIN_ARTIFACT_PREFIX)) {
+      possibleInterpreterDirectories.add(interpreterName.replace(ZEPPELIN_ARTIFACT_PREFIX, ""));
+    } else {
+      possibleInterpreterDirectories.add(ZEPPELIN_ARTIFACT_PREFIX + interpreterName);
+    }
+
+    for (String pn : possibleInterpreterDirectories) {
+      Path testInterpreterDir = Paths.get(interpreterBaseDir, pn);
+      if (Files.exists(testInterpreterDir)) {
+        throw new Exception("Interpreter " + interpreterName + " already exists with " + pn);
+      }
+    }
+
+    final Path interpreterDir = Paths.get(interpreterBaseDir, interpreterName);
+
+    try {
+      Files.createDirectories(interpreterDir);
+    } catch (Exception e) {
+      throw new Exception("Cannot create " + interpreterDir.toString());
+    }
+
+    // It might take time to finish it
+    executorService.execute(
+        new Runnable() {
+          @Override
+          public void run() {
+            downloadInterpreter(request, dependencyResolver, interpreterDir, serviceCallback);
+          }
+        });
+  }
+
+  void downloadInterpreter(
+      InterpreterInstallationRequest request,
+      DependencyResolver dependencyResolver,
+      Path interpreterDir,
+      ServiceCallback serviceCallback) {
+    try {
+      logger.info("Start to download a dependency: {}", request.getName());
+      if (null != serviceCallback) {
+        serviceCallback.onStart("Starting to download " + request.getName() + " interpreter");
+      }
+
+      dependencyResolver.load(request.getArtifact(), interpreterDir.toFile());
+      interpreterSettingManager.refreshInterpreterTemplates();
+      logger.info(
+          "Finish downloading a dependency {} into {}",
+          request.getName(),
+          interpreterDir.toString());
+      if (null != serviceCallback) {
+        serviceCallback.onSuccess(request.getName() + " downloaded");
+      }
+    } catch (RepositoryException | IOException e) {
+      logger.error("Error while downloading dependencies", e);
+      try {
+        FileUtils.deleteDirectory(interpreterDir.toFile());
+      } catch (IOException e1) {
+        logger.error(
+            "Error while removing directory. You should handle it manually: {}",
+            interpreterDir.toString(),
+            e1);
+      }
+      if (null != serviceCallback) {
+        serviceCallback.onFailure(
+            "Error while downloading " + request.getName() + " as " + e.getMessage());
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/zeppelin/blob/5fdbd533/zeppelin-server/src/main/java/org/apache/zeppelin/socket/NotebookServer.java
----------------------------------------------------------------------
diff --git a/zeppelin-server/src/main/java/org/apache/zeppelin/socket/NotebookServer.java b/zeppelin-server/src/main/java/org/apache/zeppelin/socket/NotebookServer.java
index ef43625..72fc63f 100644
--- a/zeppelin-server/src/main/java/org/apache/zeppelin/socket/NotebookServer.java
+++ b/zeppelin-server/src/main/java/org/apache/zeppelin/socket/NotebookServer.java
@@ -489,6 +489,18 @@ public class NotebookServer extends WebSocketServlet
     }
   }
 
+  public void broadcast(Message m) {
+    synchronized (connectedSockets) {
+      for (NotebookSocket ns : connectedSockets) {
+        try {
+          ns.send(serializeMessage(m));
+        } catch (IOException e) {
+          LOG.error("Send error: " + m, e);
+        }
+      }
+    }
+  }
+
   private void broadcast(String noteId, Message m) {
     List<NotebookSocket> socketsToBroadcast = Collections.emptyList();
     synchronized (noteSocketMap) {

http://git-wip-us.apache.org/repos/asf/zeppelin/blob/5fdbd533/zeppelin-server/src/main/java/org/apache/zeppelin/socket/ServiceCallback.java
----------------------------------------------------------------------
diff --git a/zeppelin-server/src/main/java/org/apache/zeppelin/socket/ServiceCallback.java b/zeppelin-server/src/main/java/org/apache/zeppelin/socket/ServiceCallback.java
new file mode 100644
index 0000000..ddfe7c4
--- /dev/null
+++ b/zeppelin-server/src/main/java/org/apache/zeppelin/socket/ServiceCallback.java
@@ -0,0 +1,27 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES 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.socket;
+
+/** This will be used by some services to pass messages to frontend via WebSocket */
+public interface ServiceCallback {
+  void onStart(String message);
+
+  void onSuccess(String message);
+
+  void onFailure(String message);
+}

http://git-wip-us.apache.org/repos/asf/zeppelin/blob/5fdbd533/zeppelin-server/src/test/java/org/apache/zeppelin/service/InterpreterServiceTest.java
----------------------------------------------------------------------
diff --git a/zeppelin-server/src/test/java/org/apache/zeppelin/service/InterpreterServiceTest.java b/zeppelin-server/src/test/java/org/apache/zeppelin/service/InterpreterServiceTest.java
new file mode 100644
index 0000000..ab74012
--- /dev/null
+++ b/zeppelin-server/src/test/java/org/apache/zeppelin/service/InterpreterServiceTest.java
@@ -0,0 +1,142 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES 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.service;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
+import static org.mockito.Mockito.doNothing;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+import java.io.IOException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import org.apache.commons.io.FileUtils;
+import org.apache.zeppelin.conf.ZeppelinConfiguration;
+import org.apache.zeppelin.dep.DependencyResolver;
+import org.apache.zeppelin.interpreter.InterpreterSettingManager;
+import org.apache.zeppelin.rest.message.InterpreterInstallationRequest;
+import org.apache.zeppelin.socket.ServiceCallback;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.mockito.Mock;
+import org.mockito.runners.MockitoJUnitRunner;
+
+@RunWith(MockitoJUnitRunner.class)
+public class InterpreterServiceTest {
+  @Mock private ZeppelinConfiguration mockZeppelinConfiguration;
+  @Mock private InterpreterSettingManager mockInterpreterSettingManager;
+
+  private Path temporaryDir;
+  private Path interpreterDir;
+  private Path localRepoDir;
+
+  InterpreterService interpreterService;
+
+  @Before
+  public void setUp() throws Exception {
+    temporaryDir = Files.createTempDirectory("tmp");
+    interpreterDir = Files.createTempDirectory(temporaryDir, "interpreter");
+    localRepoDir = Files.createTempDirectory(temporaryDir, "local-repo");
+
+    when(mockZeppelinConfiguration.getInterpreterDir()).thenReturn(interpreterDir.toString());
+    when(mockZeppelinConfiguration.getInterpreterLocalRepoPath())
+        .thenReturn(localRepoDir.toString());
+
+    when(mockZeppelinConfiguration.getZeppelinProxyUrl()).thenReturn(null);
+    when(mockZeppelinConfiguration.getZeppelinProxyUser()).thenReturn(null);
+    when(mockZeppelinConfiguration.getZeppelinProxyPassword()).thenReturn(null);
+
+    interpreterService =
+        new InterpreterService(mockZeppelinConfiguration, mockInterpreterSettingManager);
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    if (null != temporaryDir) {
+      FileUtils.deleteDirectory(temporaryDir.toFile());
+    }
+  }
+
+  @Test(expected = Exception.class)
+  public void invalidProxyUrl() throws Exception {
+    when(mockZeppelinConfiguration.getZeppelinProxyUrl()).thenReturn("invalidProxyPath");
+
+    interpreterService.installInterpreter(
+        new InterpreterInstallationRequest("name", "artifact"), null);
+  }
+
+  @Test(expected = Exception.class)
+  public void interpreterAlreadyExist() throws Exception {
+    String alreadyExistName = "aen";
+    Path specificInterpreterDir =
+        Files.createDirectory(Paths.get(interpreterDir.toString(), alreadyExistName));
+
+    interpreterService.installInterpreter(
+        new InterpreterInstallationRequest(alreadyExistName, "artifact"), null);
+  }
+
+  @Test(expected = Exception.class)
+  public void interpreterAlreadyExistWithDifferentName() throws Exception {
+    String interpreterName = "in";
+    Files.createDirectory(Paths.get(interpreterDir.toString(), interpreterName));
+
+    String anotherButSameInterpreterName = "zeppelin-" + interpreterName;
+
+    interpreterService.installInterpreter(
+        new InterpreterInstallationRequest(anotherButSameInterpreterName, "artifact"), null);
+  }
+
+  @Test
+  public void downloadInterpreter() throws IOException {
+    final String interpreterName = "test-interpreter";
+    String artifactName = "junit:junit:4.11";
+    Path specificInterpreterPath =
+        Files.createDirectory(Paths.get(interpreterDir.toString(), interpreterName));
+    DependencyResolver dependencyResolver = new DependencyResolver(localRepoDir.toString());
+
+    doNothing().when(mockInterpreterSettingManager).refreshInterpreterTemplates();
+
+    interpreterService.downloadInterpreter(
+        new InterpreterInstallationRequest(interpreterName, artifactName),
+        dependencyResolver,
+        specificInterpreterPath,
+        new ServiceCallback() {
+          @Override
+          public void onStart(String message) {
+            assertEquals("Starting to download " + interpreterName + " interpreter", message);
+          }
+
+          @Override
+          public void onSuccess(String message) {
+            assertEquals(interpreterName + " downloaded", message);
+          }
+
+          @Override
+          public void onFailure(String message) {
+            fail();
+          }
+        });
+
+    verify(mockInterpreterSettingManager, times(1)).refreshInterpreterTemplates();
+  }
+}

http://git-wip-us.apache.org/repos/asf/zeppelin/blob/5fdbd533/zeppelin-web/src/app/helium/helium.controller.js
----------------------------------------------------------------------
diff --git a/zeppelin-web/src/app/helium/helium.controller.js b/zeppelin-web/src/app/helium/helium.controller.js
index 043a9ad..81e0853 100644
--- a/zeppelin-web/src/app/helium/helium.controller.js
+++ b/zeppelin-web/src/app/helium/helium.controller.js
@@ -196,23 +196,35 @@ export default function HeliumCtrl($scope, $rootScope, $sce,
   $scope.enable = function(name, artifact, type, groupId, description) {
     let license = getLicense(name, artifact);
     let mavenArtifactInfoToHTML = groupId + ':' + artifact.split('@')[0] + ':' + artifact.split('@')[1];
-    let zeppelinVersion = $rootScope.zeppelinVersion;
-    let url = 'https://zeppelin.apache.org/docs/' + zeppelinVersion + '/manual/interpreterinstallation.html';
 
     let confirm = '';
     if (type === HeliumType.INTERPRETER) {
-      confirm = BootstrapDialog.show({
-        title: '',
-        message: '<p>Below command will download maven artifact ' +
-        '<code style="font-size: 11.5px; background-color: #f5f5f5; color: #0a0a0a">' +
-        mavenArtifactInfoToHTML + '</code>' +
-        ' and all of its transitive dependencies into interpreter/interpreter-name directory.<p>' +
-        '<div class="highlight"><pre><code class="text language-text" data-lang="text" style="font-size: 11.5px">' +
-        './bin/install-interpreter.sh --name "interpreter-name" --artifact ' +
-        mavenArtifactInfoToHTML + ' </code></pre>' +
-        '<p>After restart Zeppelin, create interpreter setting and bind it with your note. ' +
-        'For more detailed information, see <a target="_blank" href=' +
-        url + '>Interpreter Installation.</a></p>',
+      confirm = BootstrapDialog.confirm({
+        closable: false,
+        closeByBackdrop: false,
+        closeByKeyboard: false,
+        title: '<div style="font-weight: 300;">Do you want to download an interpreter?</div>',
+        message: '<div style="font-size: 14px; margin-top: 5px;">Name</div>' +
+          `<div style="color:gray">${name}</div>` +
+          '<hr style="margin-top: 10px; margin-bottom: 10px;" />' +
+          '<div style="font-size: 14px; margin-bottom: 2px;">Artifact</div>' +
+          `<div style="color:gray">${mavenArtifactInfoToHTML}</div>`,
+        callback: function(result) {
+          if (result) {
+            heliumService.installInterpreter({'name': name, 'artifact': mavenArtifactInfoToHTML})
+            .success(function(data, status) {
+              confirm.close();
+            }).error(function(data, status) {
+              confirm.close();
+              console.log('Failed to install an interpreter %o %o', name, artifact);
+              BootstrapDialog.show({
+                title: 'Error while starting to install ' + name + ' interpreter',
+                message: _.escape(data.message),
+              });
+            });
+            return false;
+          }
+        },
       });
     } else {
       confirm = BootstrapDialog.confirm({

http://git-wip-us.apache.org/repos/asf/zeppelin/blob/5fdbd533/zeppelin-web/src/app/helium/helium.service.js
----------------------------------------------------------------------
diff --git a/zeppelin-web/src/app/helium/helium.service.js b/zeppelin-web/src/app/helium/helium.service.js
index 7501fae..4f67273 100644
--- a/zeppelin-web/src/app/helium/helium.service.js
+++ b/zeppelin-web/src/app/helium/helium.service.js
@@ -106,6 +106,10 @@ export default function HeliumService($http, $sce, baseUrlSrv) {
     return $http.post(baseUrlSrv.getRestApiBase() + '/helium/disable/' + name);
   };
 
+  this.installInterpreter = function(installRequest) {
+    return $http.post(baseUrlSrv.getRestApiBase() + '/interpreter/install', installRequest);
+  };
+
   this.saveConfig = function(pkg, defaultPackageConfig, closeConfigPanelCallback) {
     // in case of local package, it will include `/`
     const pkgArtifact = encodeURIComponent(pkg.artifact);

http://git-wip-us.apache.org/repos/asf/zeppelin/blob/5fdbd533/zeppelin-web/src/components/websocket/websocket-event.factory.js
----------------------------------------------------------------------
diff --git a/zeppelin-web/src/components/websocket/websocket-event.factory.js b/zeppelin-web/src/components/websocket/websocket-event.factory.js
index ca33263..80b8807 100644
--- a/zeppelin-web/src/components/websocket/websocket-event.factory.js
+++ b/zeppelin-web/src/components/websocket/websocket-event.factory.js
@@ -14,7 +14,7 @@
 
 angular.module('zeppelinWebApp').factory('websocketEvents', WebsocketEventFactory);
 
-function WebsocketEventFactory($rootScope, $websocket, $location, baseUrlSrv) {
+function WebsocketEventFactory($rootScope, $websocket, $location, baseUrlSrv, ngToast) {
   'ngInject';
 
   let websocketCalls = {};
@@ -177,6 +177,10 @@ function WebsocketEventFactory($rootScope, $websocket, $location, baseUrlSrv) {
       $rootScope.$broadcast('setNoteRevisionResult', data);
     } else if (op === 'PARAS_INFO') {
       $rootScope.$broadcast('updateParaInfos', data);
+    } else if (op === 'INTERPRETER_INSTALL_STARTED') {
+      ngToast.info(data.message);
+    } else if (op === 'INTERPRETER_INSTALL_RESULT') {
+      ngToast.info(data.message);
     } else {
       console.error(`unknown websocket op: ${op}`);
     }

http://git-wip-us.apache.org/repos/asf/zeppelin/blob/5fdbd533/zeppelin-zengine/src/main/java/org/apache/zeppelin/interpreter/InterpreterSettingManager.java
----------------------------------------------------------------------
diff --git a/zeppelin-zengine/src/main/java/org/apache/zeppelin/interpreter/InterpreterSettingManager.java b/zeppelin-zengine/src/main/java/org/apache/zeppelin/interpreter/InterpreterSettingManager.java
index e725675..125bc18 100644
--- a/zeppelin-zengine/src/main/java/org/apache/zeppelin/interpreter/InterpreterSettingManager.java
+++ b/zeppelin-zengine/src/main/java/org/apache/zeppelin/interpreter/InterpreterSettingManager.java
@@ -21,9 +21,11 @@ import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
 import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.Maps;
+import com.google.common.collect.Sets;
 import com.google.gson.Gson;
 import com.google.gson.GsonBuilder;
 import com.google.gson.reflect.TypeToken;
+import java.util.Set;
 import org.apache.commons.io.FileUtils;
 import org.apache.commons.lang.ArrayUtils;
 import org.apache.commons.lang.StringUtils;
@@ -136,34 +138,38 @@ public class InterpreterSettingManager {
   }
 
   public InterpreterSettingManager(ZeppelinConfiguration conf,
-                                   InterpreterOption defaultOption,
-                                   AngularObjectRegistryListener angularObjectRegistryListener,
-                                   RemoteInterpreterProcessListener
-                                         remoteInterpreterProcessListener,
-                                   ApplicationEventListener appEventListener,
-                                   ConfigStorage configStorage) throws IOException {
+      InterpreterOption defaultOption,
+      AngularObjectRegistryListener angularObjectRegistryListener,
+      RemoteInterpreterProcessListener remoteInterpreterProcessListener,
+      ApplicationEventListener appEventListener,
+      ConfigStorage configStorage)
+      throws IOException {
     this.conf = conf;
     this.defaultOption = defaultOption;
     this.interpreterDirPath = Paths.get(conf.getInterpreterDir());
     LOGGER.debug("InterpreterRootPath: {}", interpreterDirPath);
-    this.dependencyResolver = new DependencyResolver(
-        conf.getString(ConfVars.ZEPPELIN_INTERPRETER_LOCALREPO));
+    this.dependencyResolver =
+        new DependencyResolver(conf.getString(ConfVars.ZEPPELIN_INTERPRETER_LOCALREPO));
     this.interpreterRepositories = dependencyResolver.getRepos();
-    this.interpreterGroupOrderList = Arrays.asList(conf.getString(
-        ConfVars.ZEPPELIN_INTERPRETER_GROUP_ORDER).split(","));
+    this.interpreterGroupOrderList =
+        Arrays.asList(conf.getString(ConfVars.ZEPPELIN_INTERPRETER_GROUP_ORDER).split(","));
     this.gson = new GsonBuilder().setPrettyPrinting().create();
 
     this.angularObjectRegistryListener = angularObjectRegistryListener;
     this.remoteInterpreterProcessListener = remoteInterpreterProcessListener;
     this.appEventListener = appEventListener;
-    this.recoveryStorage = ReflectionUtils.createClazzInstance(conf.getRecoveryStorageClass(),
-        new Class[] {ZeppelinConfiguration.class, InterpreterSettingManager.class},
-        new Object[] {conf, this});
+    this.recoveryStorage =
+        ReflectionUtils.createClazzInstance(
+            conf.getRecoveryStorageClass(),
+            new Class[] {ZeppelinConfiguration.class, InterpreterSettingManager.class},
+            new Object[] {conf, this});
     this.recoveryStorage.init();
     LOGGER.info("Using RecoveryStorage: " + this.recoveryStorage.getClass().getName());
-    this.lifecycleManager = ReflectionUtils.createClazzInstance(conf.getLifecycleManagerClass(),
-        new Class[] {ZeppelinConfiguration.class},
-        new Object[] {conf});
+    this.lifecycleManager =
+        ReflectionUtils.createClazzInstance(
+            conf.getLifecycleManagerClass(),
+            new Class[] {ZeppelinConfiguration.class},
+            new Object[] {conf});
     LOGGER.info("Using LifecycleManager: " + this.lifecycleManager.getClass().getName());
 
     this.configStorage = configStorage;
@@ -171,6 +177,22 @@ public class InterpreterSettingManager {
     init();
   }
 
+  public void refreshInterpreterTemplates() {
+    Set<String> installedInterpreters = Sets.newHashSet(interpreterSettingTemplates.keySet());
+
+    try {
+      LOGGER.info("Refreshing interpreter list");
+      loadInterpreterSettingFromDefaultDir(false);
+      Set<String> newlyAddedInterpreters = Sets.newHashSet(interpreterSettingTemplates.keySet());
+      newlyAddedInterpreters.removeAll(installedInterpreters);
+      if(!newlyAddedInterpreters.isEmpty()) {
+        saveToFile();
+      }
+    } catch (IOException e) {
+      LOGGER.error("Error while saving interpreter settings.");
+    }
+  }
+
 
   private void initInterpreterSetting(InterpreterSetting interpreterSetting) {
     interpreterSetting.setConf(conf)
@@ -304,6 +326,12 @@ public class InterpreterSettingManager {
 
   private void init() throws IOException {
 
+    loadInterpreterSettingFromDefaultDir(true);
+    loadFromFile();
+    saveToFile();
+  }
+
+  private void loadInterpreterSettingFromDefaultDir(boolean override) throws IOException {
     // 1. detect interpreter setting via interpreter-setting.json in each interpreter folder
     // 2. detect interpreter setting in interpreter.json that is saved before
     String interpreterJson = conf.getInterpreterJson();
@@ -324,8 +352,9 @@ public class InterpreterSettingManager {
          * 2. Register it from interpreter-setting.json in classpath
          *    {ZEPPELIN_HOME}/interpreter/{interpreter_name}
          */
-        if (!registerInterpreterFromPath(interpreterDirString, interpreterJson)) {
-          if (!registerInterpreterFromResource(cl, interpreterDirString, interpreterJson)) {
+        if (!registerInterpreterFromPath(interpreterDirString, interpreterJson, override)) {
+          if (!registerInterpreterFromResource(cl, interpreterDirString, interpreterJson,
+              override)) {
             LOGGER.warn("No interpreter-setting.json found in " + interpreterDirString);
           }
         }
@@ -333,9 +362,6 @@ public class InterpreterSettingManager {
     } else {
       LOGGER.warn("InterpreterDir {} doesn't exist", interpreterDirPath);
     }
-
-    loadFromFile();
-    saveToFile();
   }
 
   public RemoteInterpreterProcessListener getRemoteInterpreterProcessListener() {
@@ -347,7 +373,7 @@ public class InterpreterSettingManager {
   }
 
   private boolean registerInterpreterFromResource(ClassLoader cl, String interpreterDir,
-                                                  String interpreterJson) throws IOException {
+      String interpreterJson, boolean override) throws IOException {
     URL[] urls = recursiveBuildLibList(new File(interpreterDir));
     ClassLoader tempClassLoader = new URLClassLoader(urls, null);
 
@@ -359,19 +385,19 @@ public class InterpreterSettingManager {
     LOGGER.debug("Reading interpreter-setting.json from {} as Resource", url);
     List<RegisteredInterpreter> registeredInterpreterList =
         getInterpreterListFromJson(url.openStream());
-    registerInterpreterSetting(registeredInterpreterList, interpreterDir);
+    registerInterpreterSetting(registeredInterpreterList, interpreterDir, override);
     return true;
   }
 
-  private boolean registerInterpreterFromPath(String interpreterDir, String interpreterJson)
-      throws IOException {
+  private boolean registerInterpreterFromPath(String interpreterDir, String interpreterJson,
+      boolean override) throws IOException {
 
     Path interpreterJsonPath = Paths.get(interpreterDir, interpreterJson);
     if (Files.exists(interpreterJsonPath)) {
       LOGGER.debug("Reading interpreter-setting.json from file {}", interpreterJsonPath);
       List<RegisteredInterpreter> registeredInterpreterList =
           getInterpreterListFromJson(new FileInputStream(interpreterJsonPath.toFile()));
-      registerInterpreterSetting(registeredInterpreterList, interpreterDir);
+      registerInterpreterSetting(registeredInterpreterList, interpreterDir, override);
       return true;
     }
     return false;
@@ -384,7 +410,7 @@ public class InterpreterSettingManager {
   }
 
   private void registerInterpreterSetting(List<RegisteredInterpreter> registeredInterpreters,
-                                          String interpreterDir) throws IOException {
+      String interpreterDir, boolean override) {
 
     Map<String, DefaultInterpreterProperty> properties = new HashMap<>();
     List<InterpreterInfo> interpreterInfos = new ArrayList<>();
@@ -420,10 +446,11 @@ public class InterpreterSettingManager {
         .setIntepreterSettingManager(this)
         .create();
 
-    LOGGER.info("Register InterpreterSettingTemplate: {}",
-        interpreterSettingTemplate.getName());
-    interpreterSettingTemplates.put(interpreterSettingTemplate.getName(),
-        interpreterSettingTemplate);
+    String key = interpreterSettingTemplate.getName();
+    if(override || !interpreterSettingTemplates.containsKey(key)) {
+      LOGGER.info("Register InterpreterSettingTemplate: {}", key);
+      interpreterSettingTemplates.put(key, interpreterSettingTemplate);
+    }
   }
 
   @VisibleForTesting
@@ -687,13 +714,6 @@ public class InterpreterSettingManager {
     return setting;
   }
 
-  @VisibleForTesting
-  public void addInterpreterSetting(InterpreterSetting interpreterSetting) {
-    interpreterSettingTemplates.put(interpreterSetting.getName(), interpreterSetting);
-    initInterpreterSetting(interpreterSetting);
-    interpreterSettings.put(interpreterSetting.getId(), interpreterSetting);
-  }
-
   /**
    * map interpreter ids into noteId
    *

http://git-wip-us.apache.org/repos/asf/zeppelin/blob/5fdbd533/zeppelin-zengine/src/main/java/org/apache/zeppelin/notebook/socket/Message.java
----------------------------------------------------------------------
diff --git a/zeppelin-zengine/src/main/java/org/apache/zeppelin/notebook/socket/Message.java b/zeppelin-zengine/src/main/java/org/apache/zeppelin/notebook/socket/Message.java
index 82d96ae..2d6a153 100644
--- a/zeppelin-zengine/src/main/java/org/apache/zeppelin/notebook/socket/Message.java
+++ b/zeppelin-zengine/src/main/java/org/apache/zeppelin/notebook/socket/Message.java
@@ -185,7 +185,9 @@ public class Message implements JsonSerializable {
     RUN_PARAGRAPH_USING_SPELL,    // [s-c] run paragraph using spell
     PARAS_INFO,                   // [s-c] paragraph runtime infos
     SAVE_NOTE_FORMS,              // save note forms
-    REMOVE_NOTE_FORMS             // remove note forms
+    REMOVE_NOTE_FORMS,            // remove note forms
+    INTERPRETER_INSTALL_STARTED,  // [s-c] start to download an interpreter
+    INTERPRETER_INSTALL_RESULT    // [s-c] Status of an interpreter installation
   }
 
   private static final Gson gson = new Gson();

http://git-wip-us.apache.org/repos/asf/zeppelin/blob/5fdbd533/zeppelin-zengine/src/test/java/org/apache/zeppelin/interpreter/AbstractInterpreterTest.java
----------------------------------------------------------------------
diff --git a/zeppelin-zengine/src/test/java/org/apache/zeppelin/interpreter/AbstractInterpreterTest.java b/zeppelin-zengine/src/test/java/org/apache/zeppelin/interpreter/AbstractInterpreterTest.java
index 16c8c1d..1a6a7b1 100644
--- a/zeppelin-zengine/src/test/java/org/apache/zeppelin/interpreter/AbstractInterpreterTest.java
+++ b/zeppelin-zengine/src/test/java/org/apache/zeppelin/interpreter/AbstractInterpreterTest.java
@@ -15,7 +15,6 @@ import java.io.File;
 
 import static org.mockito.Mockito.mock;
 
-
 /**
  * This class will load configuration files under
  *   src/test/resources/interpreter

http://git-wip-us.apache.org/repos/asf/zeppelin/blob/5fdbd533/zeppelin-zengine/src/test/java/org/apache/zeppelin/interpreter/InterpreterSettingManagerTest.java
----------------------------------------------------------------------
diff --git a/zeppelin-zengine/src/test/java/org/apache/zeppelin/interpreter/InterpreterSettingManagerTest.java b/zeppelin-zengine/src/test/java/org/apache/zeppelin/interpreter/InterpreterSettingManagerTest.java
index 444f366..1d3e2ff 100644
--- a/zeppelin-zengine/src/test/java/org/apache/zeppelin/interpreter/InterpreterSettingManagerTest.java
+++ b/zeppelin-zengine/src/test/java/org/apache/zeppelin/interpreter/InterpreterSettingManagerTest.java
@@ -35,7 +35,6 @@ import java.util.List;
 import java.util.Map;
 
 import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotEquals;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;