You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@zeppelin.apache.org by bz...@apache.org on 2016/06/02 22:24:32 UTC

incubator-zeppelin git commit: [ZEPPELIN-828] Use individual user credentials for data source authentication

Repository: incubator-zeppelin
Updated Branches:
  refs/heads/master 5ef600c74 -> bdc78da0e


[ZEPPELIN-828] Use individual user credentials for data source authentication

### What is this PR for?
The purpose of this pull request is to allow users to use their own credentials to authenticate with data sources.
It contains the following:
- web UI and rest API to let users input their credentials
- changes to AuthenticationInfo to add data source credentials
- changes to Paragraph to set data source credentials in AuthenticationInfo
- changes to HiveInterpreter to use the user credentials and simplification to use a new connection for every query. We made this change since we found an issue with closed vertica connections. Since we are using HiveInterpreter for analytics, the cost of creating a new connection is small compared to query execution. We don't have to merge this change into master.

### What type of PR is it?
Feature

### Todos
* [x] - Save user credentials in a file
* [x] - Do not persist AuthenticationInfo in paragraph since it contains sensitive information
* [ ] - Add tests
* [ ] - Add documentation

### What is the Jira issue?
[ZEPPELIN-828] (https://issues.apache.org/jira/browse/ZEPPELIN-828)

### How should this be tested?
1. In conf/shiro.ini, use /** = authc
1. Login to Zeppelin
2. Enter credentials for a data source
3. Check if the interpreter is using the correct credentials

### Screenshots (if appropriate)
https://gfycat.com/LiquidGentleKatydid

### Questions:
* Does this needs documentation? Yes.

Author: Prasad Wagle <pw...@twitter.com>

Closes #860 from prasadwagle/ZEPPELIN-828 and squashes the following commits:

f6d8de3 [Prasad Wagle] Use LOG.error(msg, e)
62b64a0 [Prasad Wagle] Use LOG.error(msg, e)
c628099 [Prasad Wagle] Use a more general term 'entity' instead of 'datasource'
cbbb591 [Prasad Wagle] Added conf/credentials.json to .gitignore
623d42a [Prasad Wagle] Revert changes to HiveInterpreter, Update tests
f2628e3 [Prasad Wagle] Remove vertica dependency from hive/pom.xml
d4e3cd1 [Prasad Wagle] Fix checkstyle error, Update Data Source Authorization section in docs
8b70f08 [Prasad Wagle] Persist credentials depending on conf setting
d9dae04 [Prasad Wagle] Persist user info instead of authentication info in paragraph
cfe4c86 [Prasad Wagle] Pass all user credentials to the interpreter
90d546f [Prasad Wagle] Use individual user credentials for data source authentication


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

Branch: refs/heads/master
Commit: bdc78da0e6e4fc73a8992379286ac7319b028b57
Parents: 5ef600c
Author: Prasad Wagle <pw...@twitter.com>
Authored: Wed Jun 1 09:50:43 2016 -0700
Committer: Alexander Bezzubov <bz...@apache.org>
Committed: Fri Jun 3 07:24:21 2016 +0900

----------------------------------------------------------------------
 .gitignore                                      |   1 +
 docs/security/interpreter_authorization.md      |   2 +
 pom.xml                                         |   1 +
 .../zeppelin/user/AuthenticationInfo.java       |  10 ++
 .../org/apache/zeppelin/user/Credentials.java   | 122 +++++++++++++++++++
 .../zeppelin/user/CredentialsInfoSaving.java    |  27 ++++
 .../apache/zeppelin/user/UserCredentials.java   |  47 +++++++
 .../apache/zeppelin/user/UsernamePassword.java  |  55 +++++++++
 .../apache/zeppelin/user/CredentialsTest.java   |  40 ++++++
 .../apache/zeppelin/rest/CredentialRestApi.java |  78 ++++++++++++
 .../apache/zeppelin/server/ZeppelinServer.java  |   8 +-
 zeppelin-web/src/app/app.js                     |   4 +
 .../src/app/credential/credential.controller.js |  40 ++++++
 zeppelin-web/src/app/credential/credential.css  |  86 +++++++++++++
 zeppelin-web/src/app/credential/credential.html |  71 +++++++++++
 .../notebook/paragraph/paragraph.controller.js  |   7 +-
 zeppelin-web/src/components/navbar/navbar.html  |   3 +
 zeppelin-web/src/index.html                     |   2 +
 .../zeppelin/conf/ZeppelinConfiguration.java    |  10 ++
 .../java/org/apache/zeppelin/notebook/Note.java |  14 ++-
 .../org/apache/zeppelin/notebook/Notebook.java  |   9 +-
 .../notebook/NotebookAuthorization.java         |   3 +-
 .../org/apache/zeppelin/notebook/Paragraph.java |  16 ++-
 .../apache/zeppelin/notebook/NotebookTest.java  |  14 ++-
 .../notebook/repo/NotebookRepoSyncTest.java     |   9 +-
 .../notebook/repo/VFSNotebookRepoTest.java      |   2 +-
 .../zeppelin/search/LuceneSearchTest.java       |   2 +-
 27 files changed, 664 insertions(+), 19 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-zeppelin/blob/bdc78da0/.gitignore
----------------------------------------------------------------------
diff --git a/.gitignore b/.gitignore
index 5328914..e13f0e9 100644
--- a/.gitignore
+++ b/.gitignore
@@ -26,6 +26,7 @@ conf/truststore
 conf/interpreter.json
 conf/notebook-authorization.json
 conf/shiro.ini
+conf/credentials.json
 
 # other generated files
 spark/dependency-reduced-pom.xml

http://git-wip-us.apache.org/repos/asf/incubator-zeppelin/blob/bdc78da0/docs/security/interpreter_authorization.md
----------------------------------------------------------------------
diff --git a/docs/security/interpreter_authorization.md b/docs/security/interpreter_authorization.md
index d1c792d..862ef9b 100644
--- a/docs/security/interpreter_authorization.md
+++ b/docs/security/interpreter_authorization.md
@@ -32,3 +32,5 @@ The interpret method takes the user string as parameter and executes the jdbc ca
 
 In case of Presto, we don't need password if the Presto DB server runs backend code using HDFS authorization for the user.
 For databases like Vertica and Mysql we have to store password information for users.
+
+The Credentials tab in the navbar allows users to save credentials for data sources which are passed to interpreters.
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-zeppelin/blob/bdc78da0/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 598f9a3..8799ff6 100755
--- a/pom.xml
+++ b/pom.xml
@@ -471,6 +471,7 @@
               <exclude>**/zeppelin-distribution/src/bin_license/**</exclude>
               <exclude>conf/interpreter.json</exclude>
               <exclude>conf/notebook-authorization.json</exclude>
+              <exclude>conf/credentials.json</exclude>
               <exclude>conf/zeppelin-env.sh</exclude>
               <exclude>spark-*-bin*/**</exclude>
               <exclude>.spark-dist/**</exclude>

http://git-wip-us.apache.org/repos/asf/incubator-zeppelin/blob/bdc78da0/zeppelin-interpreter/src/main/java/org/apache/zeppelin/user/AuthenticationInfo.java
----------------------------------------------------------------------
diff --git a/zeppelin-interpreter/src/main/java/org/apache/zeppelin/user/AuthenticationInfo.java b/zeppelin-interpreter/src/main/java/org/apache/zeppelin/user/AuthenticationInfo.java
index 5d54342..c4cd441 100644
--- a/zeppelin-interpreter/src/main/java/org/apache/zeppelin/user/AuthenticationInfo.java
+++ b/zeppelin-interpreter/src/main/java/org/apache/zeppelin/user/AuthenticationInfo.java
@@ -24,6 +24,7 @@ package org.apache.zeppelin.user;
 public class AuthenticationInfo {
   String user;
   String ticket;
+  UserCredentials userCredentials;
 
   public AuthenticationInfo() {}
 
@@ -52,4 +53,13 @@ public class AuthenticationInfo {
   public void setTicket(String ticket) {
     this.ticket = ticket;
   }
+
+  public UserCredentials getUserCredentials() {
+    return userCredentials;
+  }
+
+  public void setUserCredentials(UserCredentials userCredentials) {
+    this.userCredentials = userCredentials;
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-zeppelin/blob/bdc78da0/zeppelin-interpreter/src/main/java/org/apache/zeppelin/user/Credentials.java
----------------------------------------------------------------------
diff --git a/zeppelin-interpreter/src/main/java/org/apache/zeppelin/user/Credentials.java b/zeppelin-interpreter/src/main/java/org/apache/zeppelin/user/Credentials.java
new file mode 100644
index 0000000..b3a3726
--- /dev/null
+++ b/zeppelin-interpreter/src/main/java/org/apache/zeppelin/user/Credentials.java
@@ -0,0 +1,122 @@
+/*
+* Licensed to the Apache Software Foundation (ASF) under one or more
+* contributor license agreements.  See the NOTICE file distributed with
+* this work for additional information regarding copyright ownership.
+* The ASF licenses this file to You under the Apache License, Version 2.0
+* (the "License"); you may not use this file except in compliance with
+* the License.  You may obtain a copy of the License at
+*
+*    http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES 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.user;
+
+
+import com.google.gson.Gson;
+import com.google.gson.GsonBuilder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.*;
+import java.util.HashMap;
+import java.util.Map;
+
+/**
+ * Class defining credentials for data source authorization
+ */
+public class Credentials {
+  private static final Logger LOG = LoggerFactory.getLogger(Credentials.class);
+
+  private Map<String, UserCredentials> credentialsMap;
+  private Gson gson;
+  private Boolean credentialsPersist = true;
+  File credentialsFile;
+
+  public Credentials(Boolean credentialsPersist, String credentialsPath) {
+    this.credentialsPersist = credentialsPersist;
+    if (credentialsPath != null) {
+      credentialsFile = new File(credentialsPath);
+    }
+    credentialsMap = new HashMap<>();
+    if (credentialsPersist) {
+      GsonBuilder builder = new GsonBuilder();
+      builder.setPrettyPrinting();
+      gson = builder.create();
+      loadFromFile();
+    }
+  }
+
+  public UserCredentials getUserCredentials(String username) {
+    UserCredentials uc = credentialsMap.get(username);
+    if (uc == null) {
+      uc = new UserCredentials();
+    }
+    return uc;
+  }
+
+  public void putUserCredentials(String username, UserCredentials uc) throws IOException {
+    credentialsMap.put(username, uc);
+    if (credentialsPersist) {
+      saveToFile();
+    }
+  }
+
+  private void loadFromFile() {
+    LOG.info(credentialsFile.getAbsolutePath());
+    if (!credentialsFile.exists()) {
+      // nothing to read
+      return;
+    }
+
+    try {
+      FileInputStream fis = new FileInputStream(credentialsFile);
+      InputStreamReader isr = new InputStreamReader(fis);
+      BufferedReader bufferedReader = new BufferedReader(isr);
+      StringBuilder sb = new StringBuilder();
+      String line;
+      while ((line = bufferedReader.readLine()) != null) {
+        sb.append(line);
+      }
+      isr.close();
+      fis.close();
+
+      String json = sb.toString();
+      CredentialsInfoSaving info = gson.fromJson(json, CredentialsInfoSaving.class);
+      this.credentialsMap = info.credentialsMap;
+    } catch (IOException e) {
+      LOG.error("Error loading credentials file", e);
+      e.printStackTrace();
+    }
+  }
+
+  private void saveToFile() throws IOException {
+    String jsonString;
+
+    synchronized (credentialsMap) {
+      CredentialsInfoSaving info = new CredentialsInfoSaving();
+      info.credentialsMap = credentialsMap;
+      jsonString = gson.toJson(info);
+    }
+
+    try {
+      if (!credentialsFile.exists()) {
+        credentialsFile.createNewFile();
+      }
+
+      FileOutputStream fos = new FileOutputStream(credentialsFile, false);
+      OutputStreamWriter out = new OutputStreamWriter(fos);
+      out.append(jsonString);
+      out.close();
+      fos.close();
+    } catch (IOException e) {
+      LOG.error("Error saving credentials file", e);
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-zeppelin/blob/bdc78da0/zeppelin-interpreter/src/main/java/org/apache/zeppelin/user/CredentialsInfoSaving.java
----------------------------------------------------------------------
diff --git a/zeppelin-interpreter/src/main/java/org/apache/zeppelin/user/CredentialsInfoSaving.java b/zeppelin-interpreter/src/main/java/org/apache/zeppelin/user/CredentialsInfoSaving.java
new file mode 100644
index 0000000..20b586c
--- /dev/null
+++ b/zeppelin-interpreter/src/main/java/org/apache/zeppelin/user/CredentialsInfoSaving.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.user;
+
+import java.util.Map;
+
+/**
+ * Helper class to save credentials
+ */
+public class CredentialsInfoSaving {
+  public Map<String, UserCredentials> credentialsMap;
+}

http://git-wip-us.apache.org/repos/asf/incubator-zeppelin/blob/bdc78da0/zeppelin-interpreter/src/main/java/org/apache/zeppelin/user/UserCredentials.java
----------------------------------------------------------------------
diff --git a/zeppelin-interpreter/src/main/java/org/apache/zeppelin/user/UserCredentials.java b/zeppelin-interpreter/src/main/java/org/apache/zeppelin/user/UserCredentials.java
new file mode 100644
index 0000000..166840a
--- /dev/null
+++ b/zeppelin-interpreter/src/main/java/org/apache/zeppelin/user/UserCredentials.java
@@ -0,0 +1,47 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.zeppelin.user;
+
+import java.util.HashMap;
+import java.util.Map;
+
+/**
+ * User Credentials POJO
+ */
+public class UserCredentials {
+  private Map<String, UsernamePassword> userCredentials;
+
+  public UserCredentials() {
+    this.userCredentials = new HashMap<>();
+  }
+
+  public UsernamePassword getUsernamePassword(String entity) {
+    return userCredentials.get(entity);
+  }
+
+  public void putUsernamePassword(String entity, UsernamePassword up) {
+    userCredentials.put(entity, up);
+  }
+
+  @Override
+  public String toString() {
+    return "UserCredentials{" +
+        "userCredentials=" + userCredentials +
+        '}';
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-zeppelin/blob/bdc78da0/zeppelin-interpreter/src/main/java/org/apache/zeppelin/user/UsernamePassword.java
----------------------------------------------------------------------
diff --git a/zeppelin-interpreter/src/main/java/org/apache/zeppelin/user/UsernamePassword.java b/zeppelin-interpreter/src/main/java/org/apache/zeppelin/user/UsernamePassword.java
new file mode 100644
index 0000000..00116a9
--- /dev/null
+++ b/zeppelin-interpreter/src/main/java/org/apache/zeppelin/user/UsernamePassword.java
@@ -0,0 +1,55 @@
+/*
+* Licensed to the Apache Software Foundation (ASF) under one or more
+* contributor license agreements.  See the NOTICE file distributed with
+* this work for additional information regarding copyright ownership.
+* The ASF licenses this file to You under the Apache License, Version 2.0
+* (the "License"); you may not use this file except in compliance with
+* the License.  You may obtain a copy of the License at
+*
+*    http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES 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.user;
+
+/**
+ * Username and Password POJO
+ */
+public class UsernamePassword {
+  private String username;
+  private String password;
+
+  public UsernamePassword(String username, String password) {
+    this.username = username;
+    this.password = password;
+  }
+
+  public String getUsername() {
+    return username;
+  }
+
+  public void setUsername(String username) {
+    this.username = username;
+  }
+
+  public String getPassword() {
+    return password;
+  }
+
+  public void setPassword(String password) {
+    this.password = password;
+  }
+
+  @Override
+  public String toString() {
+    return "UsernamePassword{" +
+        "username='" + username + '\'' +
+        ", password='" + password + '\'' +
+        '}';
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-zeppelin/blob/bdc78da0/zeppelin-interpreter/src/test/java/org/apache/zeppelin/user/CredentialsTest.java
----------------------------------------------------------------------
diff --git a/zeppelin-interpreter/src/test/java/org/apache/zeppelin/user/CredentialsTest.java b/zeppelin-interpreter/src/test/java/org/apache/zeppelin/user/CredentialsTest.java
new file mode 100644
index 0000000..259516f
--- /dev/null
+++ b/zeppelin-interpreter/src/test/java/org/apache/zeppelin/user/CredentialsTest.java
@@ -0,0 +1,40 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.zeppelin.user;
+
+import static org.junit.Assert.*;
+
+import org.junit.Test;
+
+import java.io.IOException;
+
+public class CredentialsTest {
+
+  @Test
+  public void testDefaultProperty() throws IOException {
+    Credentials credentials = new Credentials(false, null);
+    UserCredentials userCredentials = new UserCredentials();
+    UsernamePassword up1 = new UsernamePassword("user2", "password");
+    userCredentials.putUsernamePassword("hive(vertica)", up1);
+    credentials.putUserCredentials("user1", userCredentials);
+    UserCredentials uc2 = credentials.getUserCredentials("user1");
+    UsernamePassword up2 = uc2.getUsernamePassword("hive(vertica)");
+    assertEquals(up1.getUsername(), up2.getUsername());
+    assertEquals(up1.getPassword(), up2.getPassword());
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-zeppelin/blob/bdc78da0/zeppelin-server/src/main/java/org/apache/zeppelin/rest/CredentialRestApi.java
----------------------------------------------------------------------
diff --git a/zeppelin-server/src/main/java/org/apache/zeppelin/rest/CredentialRestApi.java b/zeppelin-server/src/main/java/org/apache/zeppelin/rest/CredentialRestApi.java
new file mode 100644
index 0000000..b1a4d17
--- /dev/null
+++ b/zeppelin-server/src/main/java/org/apache/zeppelin/rest/CredentialRestApi.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.rest;
+
+import com.google.gson.Gson;
+import com.google.gson.reflect.TypeToken;
+import org.apache.zeppelin.user.Credentials;
+import org.apache.zeppelin.user.UserCredentials;
+import org.apache.zeppelin.user.UsernamePassword;
+import org.apache.zeppelin.server.JsonResponse;
+import org.apache.zeppelin.utils.SecurityUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.servlet.http.HttpServletRequest;
+import javax.ws.rs.*;
+import javax.ws.rs.core.Context;
+import javax.ws.rs.core.Response;
+import javax.ws.rs.core.Response.Status;
+import java.io.IOException;
+import java.util.Map;
+
+/**
+ * Credential Rest API
+ *
+ */
+@Path("/credential")
+@Produces("application/json")
+public class CredentialRestApi {
+  Logger logger = LoggerFactory.getLogger(CredentialRestApi.class);
+  private Credentials credentials;
+  private Gson gson = new Gson();
+
+  @Context
+  private HttpServletRequest servReq;
+
+  public CredentialRestApi() {
+
+  }
+
+  public CredentialRestApi(Credentials credentials) {
+    this.credentials = credentials;
+  }
+
+  /**
+   * Update credentials for current user
+   */
+  @PUT
+  public Response putCredentials(String message) throws IOException {
+    Map<String, String> messageMap = gson.fromJson(message,
+            new TypeToken<Map<String, String>>(){}.getType());
+    String entity = messageMap.get("entity");
+    String username = messageMap.get("username");
+    String password = messageMap.get("password");
+    String user = SecurityUtils.getPrincipal();
+    logger.info("Update credentials for user {} entity {}", user, entity);
+    UserCredentials uc = credentials.getUserCredentials(user);
+    uc.putUsernamePassword(entity, new UsernamePassword(username, password));
+    credentials.putUserCredentials(user, uc);
+    return new JsonResponse(Status.OK, "", "").build();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-zeppelin/blob/bdc78da0/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 c8fc60a..7412611 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
@@ -31,6 +31,7 @@ import org.apache.zeppelin.scheduler.SchedulerFactory;
 import org.apache.zeppelin.search.LuceneSearch;
 import org.apache.zeppelin.search.SearchService;
 import org.apache.zeppelin.socket.NotebookServer;
+import org.apache.zeppelin.user.Credentials;
 import org.eclipse.jetty.http.HttpVersion;
 import org.eclipse.jetty.server.*;
 import org.eclipse.jetty.server.handler.ContextHandlerCollection;
@@ -67,6 +68,7 @@ public class ZeppelinServer extends Application {
   private NotebookRepo notebookRepo;
   private SearchService notebookIndex;
   private NotebookAuthorization notebookAuthorization;
+  private Credentials credentials;
   private DependencyResolver depResolver;
 
   public ZeppelinServer() throws Exception {
@@ -80,9 +82,10 @@ public class ZeppelinServer extends Application {
     this.notebookRepo = new NotebookRepoSync(conf);
     this.notebookIndex = new LuceneSearch();
     this.notebookAuthorization = new NotebookAuthorization(conf);
+    this.credentials = new Credentials(conf.credentialsPersist(), conf.getCredentialsPath());
     notebook = new Notebook(conf,
         notebookRepo, schedulerFactory, replFactory, notebookWsServer,
-            notebookIndex, notebookAuthorization);
+            notebookIndex, notebookAuthorization, credentials);
   }
 
   public static void main(String[] args) throws InterruptedException {
@@ -292,6 +295,9 @@ public class ZeppelinServer extends Application {
     InterpreterRestApi interpreterApi = new InterpreterRestApi(replFactory);
     singletons.add(interpreterApi);
 
+    CredentialRestApi credentialApi = new CredentialRestApi(credentials);
+    singletons.add(credentialApi);
+
     SecurityRestApi securityApi = new SecurityRestApi();
     singletons.add(securityApi);
 

http://git-wip-us.apache.org/repos/asf/incubator-zeppelin/blob/bdc78da0/zeppelin-web/src/app/app.js
----------------------------------------------------------------------
diff --git a/zeppelin-web/src/app/app.js b/zeppelin-web/src/app/app.js
index 16fad36..bc8e52c 100644
--- a/zeppelin-web/src/app/app.js
+++ b/zeppelin-web/src/app/app.js
@@ -66,6 +66,10 @@
                     templateUrl: 'app/interpreter/interpreter.html',
                     controller: 'InterpreterCtrl'
                 })
+                .when('/credential', {
+                    templateUrl: 'app/credential/credential.html',
+                    controller: 'CredentialCtrl'
+                })
                 .when('/configuration', {
                   templateUrl: 'app/configuration/configuration.html',
                   controller: 'ConfigurationCtrl'

http://git-wip-us.apache.org/repos/asf/incubator-zeppelin/blob/bdc78da0/zeppelin-web/src/app/credential/credential.controller.js
----------------------------------------------------------------------
diff --git a/zeppelin-web/src/app/credential/credential.controller.js b/zeppelin-web/src/app/credential/credential.controller.js
new file mode 100644
index 0000000..4bb89f0
--- /dev/null
+++ b/zeppelin-web/src/app/credential/credential.controller.js
@@ -0,0 +1,40 @@
+/* jshint loopfunc: true */
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+'use strict';
+
+angular.module('zeppelinWebApp').controller('CredentialCtrl', function($scope, $route, $routeParams, $location, $rootScope,
+                                                                         $http, baseUrlSrv) {
+  $scope._ = _;
+
+  $scope.updateCredentials = function() {
+    $http.put(baseUrlSrv.getRestApiBase() + '/credential',
+      { 'entity': $scope.credentialEntity,
+        'username': $scope.credentialUsername,
+        'password': $scope.credentialPassword
+      } ).
+    success(function (data, status, headers, config) {
+      alert('Successfully saved credentials');
+      $scope.credentialEntity = '';
+      $scope.credentialUsername = '';
+      $scope.credentialPassword = '';
+      console.log('Success %o %o', status, data.message);
+    }).
+    error(function (data, status, headers, config) {
+      alert('Error saving credentials');
+      console.log('Error %o %o', status, data.message);
+    });
+  };
+
+});

http://git-wip-us.apache.org/repos/asf/incubator-zeppelin/blob/bdc78da0/zeppelin-web/src/app/credential/credential.css
----------------------------------------------------------------------
diff --git a/zeppelin-web/src/app/credential/credential.css b/zeppelin-web/src/app/credential/credential.css
new file mode 100644
index 0000000..b6fe133
--- /dev/null
+++ b/zeppelin-web/src/app/credential/credential.css
@@ -0,0 +1,86 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+.interpreterHead {
+  margin: -10px -10px 20px;
+  padding: 10px 15px 15px 15px;
+  background: white;
+  box-shadow: 0 2px 4px rgba(0, 0, 0, 0.15);
+  border-bottom: 1px solid #E5E5E5;
+}
+
+.interpreterHead .header {
+  font-family: 'Roboto', sans-serif;
+}
+
+.interpreterHead textarea,
+.interpreter textarea {
+  width: 100%;
+  display: block;
+  height: 20px;
+  resize: none;
+  border: 1px solid #CCCCCC;
+  font-size: 12px;
+}
+
+.interpreter .interpreter-title {
+  font-size: 20px;
+  font-weight: bold;
+  color: #3071a9;
+  float: left;
+  margin-top: 0;
+}
+
+.interpreter ul {
+  margin: 0;
+  padding: 0;
+}
+
+.interpreter .interpreterInfo {
+  list-style-type: none;
+}
+
+.interpreter table tr .interpreterPropertyKey {
+  padding : 5px 5px 5px 5px;
+}
+
+.interpreter table tr .interpreterPropertyValue {
+  padding : 5px 5px 5px 5px;
+  display: block;
+  max-height: 100px;
+  overflow-y: auto;
+}
+
+.interpreterSettingAdd {
+  margin : 5px 5px 5px 5px;
+  padding : 10px 10px 10px 10px;
+}
+
+.editable-wrap {
+  width : 100%;
+}
+
+.interpreter h5 {
+  font-weight: bold;
+}
+
+.new_h3 {
+  margin-top: 1px;
+  padding-top: 7px;
+  float: left;
+}
+
+.empty-properties-message {
+  color: #666;
+}

http://git-wip-us.apache.org/repos/asf/incubator-zeppelin/blob/bdc78da0/zeppelin-web/src/app/credential/credential.html
----------------------------------------------------------------------
diff --git a/zeppelin-web/src/app/credential/credential.html b/zeppelin-web/src/app/credential/credential.html
new file mode 100644
index 0000000..338845f
--- /dev/null
+++ b/zeppelin-web/src/app/credential/credential.html
@@ -0,0 +1,71 @@
+<!--
+Licensed under the Apache License, Version 2.0 (the "License");
+you may not use this file except in compliance with the License.
+You may obtain a copy of the License at
+
+    http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS,
+WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+See the License for the specific language governing permissions and
+limitations under the License.
+-->
+<div class="interpreterHead">
+  <div class="header">
+    <div class="row">
+      <div class="col-md-12">
+        <h3 class="new_h3">
+          Credentials
+        </h3>
+      </div>
+    </div>
+    <div class="row">
+      <div class="col-md-12">
+        Add credentials for entities one at a time.<br>
+      </div>
+    </div>
+  </div>
+</div>
+
+<div class="box width-full home"
+     >
+  <div>
+    <div class="row interpreter">
+    <div class="col-md-12">
+        <table class="table table-striped">
+          <thead>
+            <tr>
+              <th style="width:30%">Entity</th>
+              <th>Username</th>
+              <th>Password</th>
+              <th ng-if="valueform.$visible">action</th>
+            </tr>
+          </thead>
+          <tr>
+            <td>
+              <textarea msd-elastic ng-model="credentialEntity"></textarea>
+            </td>
+            <td>
+              <textarea msd-elastic ng-model="credentialUsername"></textarea>
+            </td>
+            <td>
+              <input type="password" ng-model="credentialPassword"/>
+            </td>
+          </tr>
+        </table>
+        <form editable-form name="valueform" onaftersave="updateCredentials()">
+          <button type="submit" class="btn btn-primary"
+                  ng-disabled="valueform.$waiting">
+            Save
+          </button>
+          <button type="button" class="btn btn-default"
+                  ng-disabled="valueform.$waiting"
+                  ng-click="valueform.$cancel()">
+            Cancel
+          </button>
+        </form>
+      </div>
+    </div>
+  </div>
+</div>

http://git-wip-us.apache.org/repos/asf/incubator-zeppelin/blob/bdc78da0/zeppelin-web/src/app/notebook/paragraph/paragraph.controller.js
----------------------------------------------------------------------
diff --git a/zeppelin-web/src/app/notebook/paragraph/paragraph.controller.js b/zeppelin-web/src/app/notebook/paragraph/paragraph.controller.js
index 110d8fa..3c86737 100644
--- a/zeppelin-web/src/app/notebook/paragraph/paragraph.controller.js
+++ b/zeppelin-web/src/app/notebook/paragraph/paragraph.controller.js
@@ -388,8 +388,8 @@ angular.module('zeppelinWebApp')
       }
 
       /** push the rest */
-      $scope.paragraph.authenticationInfo = data.paragraph.authenticationInfo;
       $scope.paragraph.aborted = data.paragraph.aborted;
+      $scope.paragraph.user = data.paragraph.user;
       $scope.paragraph.dateUpdated = data.paragraph.dateUpdated;
       $scope.paragraph.dateCreated = data.paragraph.dateCreated;
       $scope.paragraph.dateFinished = data.paragraph.dateFinished;
@@ -970,10 +970,7 @@ angular.module('zeppelinWebApp')
       }
       return '';
     }
-    var user = 'anonymous';
-    if (pdata.authenticationInfo !== null && !isEmpty(pdata.authenticationInfo.user)) {
-      user = pdata.authenticationInfo.user;
-    }
+    var user = (pdata.user === undefined || pdata.user === null) ? 'anonymous' : pdata.user;
     var desc = 'Took ' +
       moment.duration(moment(pdata.dateFinished).diff(moment(pdata.dateStarted))).humanize() +
       '. Last updated by ' + user + ' at ' + moment(pdata.dateUpdated).format('MMMM DD YYYY, h:mm:ss A') + '.';

http://git-wip-us.apache.org/repos/asf/incubator-zeppelin/blob/bdc78da0/zeppelin-web/src/components/navbar/navbar.html
----------------------------------------------------------------------
diff --git a/zeppelin-web/src/components/navbar/navbar.html b/zeppelin-web/src/components/navbar/navbar.html
index 361149e..148d67d 100644
--- a/zeppelin-web/src/components/navbar/navbar.html
+++ b/zeppelin-web/src/components/navbar/navbar.html
@@ -53,6 +53,9 @@ limitations under the License.
           <a href="#/interpreter">Interpreter</a>
         </li>
         <li>
+          <a href="#/credential">Credential</a>
+        </li>
+        <li>
           <a href="#/configuration">Configuration</a>
         </li>
       </ul>

http://git-wip-us.apache.org/repos/asf/incubator-zeppelin/blob/bdc78da0/zeppelin-web/src/index.html
----------------------------------------------------------------------
diff --git a/zeppelin-web/src/index.html b/zeppelin-web/src/index.html
index a634fbf..8efce04 100644
--- a/zeppelin-web/src/index.html
+++ b/zeppelin-web/src/index.html
@@ -55,6 +55,7 @@ limitations under the License.
     <link rel="stylesheet" href="app/notebook/notebook.css">
     <link rel="stylesheet" href="app/notebook/paragraph/paragraph.css">
     <link rel="stylesheet" href="app/interpreter/interpreter.css">
+    <link rel="stylesheet" href="app/credential/credential.css">
     <link rel="stylesheet" href="app/configuration/configuration.css">
     <link rel="stylesheet" href="fonts/font-awesome.min.css">
     <link rel="stylesheet" href="fonts/simple-line-icons.css">
@@ -144,6 +145,7 @@ limitations under the License.
     <script src="app/home/home.controller.js"></script>
     <script src="app/notebook/notebook.controller.js"></script>
     <script src="app/interpreter/interpreter.controller.js"></script>
+    <script src="app/credential/credential.controller.js"></script>
     <script src="app/configuration/configuration.controller.js"></script>
     <script src="app/notebook/paragraph/paragraph.controller.js"></script>
     <script src="app/search/result-list.controller.js"></script>

http://git-wip-us.apache.org/repos/asf/incubator-zeppelin/blob/bdc78da0/zeppelin-zengine/src/main/java/org/apache/zeppelin/conf/ZeppelinConfiguration.java
----------------------------------------------------------------------
diff --git a/zeppelin-zengine/src/main/java/org/apache/zeppelin/conf/ZeppelinConfiguration.java b/zeppelin-zengine/src/main/java/org/apache/zeppelin/conf/ZeppelinConfiguration.java
index a07c3fa..fa52957 100644
--- a/zeppelin-zengine/src/main/java/org/apache/zeppelin/conf/ZeppelinConfiguration.java
+++ b/zeppelin-zengine/src/main/java/org/apache/zeppelin/conf/ZeppelinConfiguration.java
@@ -362,6 +362,14 @@ public class ZeppelinConfiguration extends XMLConfiguration {
     return getRelativeDir(String.format("%s/notebook-authorization.json", getConfDir()));
   }
 
+  public Boolean credentialsPersist() {
+    return getBoolean(ConfVars.ZEPPELIN_CREDENTIALS_PERSIST);
+  }
+
+  public String getCredentialsPath() {
+    return getRelativeDir(String.format("%s/credentials.json", getConfDir()));
+  }
+
   public String getShiroPath() {
     return getRelativeDir(String.format("%s/shiro.ini", getConfDir()));
   }
@@ -528,8 +536,10 @@ public class ZeppelinConfiguration extends XMLConfiguration {
     // i.e. http://localhost:8080
     ZEPPELIN_ALLOWED_ORIGINS("zeppelin.server.allowed.origins", "*"),
     ZEPPELIN_ANONYMOUS_ALLOWED("zeppelin.anonymous.allowed", true),
+    ZEPPELIN_CREDENTIALS_PERSIST("zeppelin.credentials.persist", true),
     ZEPPELIN_WEBSOCKET_MAX_TEXT_MESSAGE_SIZE("zeppelin.websocket.max.text.message.size", "1024000");
 
+
     private String varName;
     @SuppressWarnings("rawtypes")
     private Class varClass;

http://git-wip-us.apache.org/repos/asf/incubator-zeppelin/blob/bdc78da0/zeppelin-zengine/src/main/java/org/apache/zeppelin/notebook/Note.java
----------------------------------------------------------------------
diff --git a/zeppelin-zengine/src/main/java/org/apache/zeppelin/notebook/Note.java b/zeppelin-zengine/src/main/java/org/apache/zeppelin/notebook/Note.java
index a73aad9..6941dc9 100644
--- a/zeppelin-zengine/src/main/java/org/apache/zeppelin/notebook/Note.java
+++ b/zeppelin-zengine/src/main/java/org/apache/zeppelin/notebook/Note.java
@@ -40,6 +40,7 @@ import org.apache.zeppelin.search.SearchService;
 
 import com.google.gson.Gson;
 import org.apache.zeppelin.user.AuthenticationInfo;
+import org.apache.zeppelin.user.Credentials;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -70,6 +71,7 @@ public class Note implements Serializable, JobListener {
   private transient NotebookRepo repo;
   private transient SearchService index;
   private transient ScheduledFuture delayedPersist;
+  private transient Credentials credentials;
 
   /**
    * note configurations.
@@ -89,11 +91,12 @@ public class Note implements Serializable, JobListener {
   public Note() {}
 
   public Note(NotebookRepo repo, NoteInterpreterLoader replLoader,
-      JobListenerFactory jlFactory, SearchService noteIndex) {
+      JobListenerFactory jlFactory, SearchService noteIndex, Credentials credentials) {
     this.repo = repo;
     this.replLoader = replLoader;
     this.jobListenerFactory = jlFactory;
     this.index = noteIndex;
+    this.credentials = credentials;
     generateId();
   }
 
@@ -145,6 +148,15 @@ public class Note implements Serializable, JobListener {
     this.index = index;
   }
 
+  public Credentials getCredentials() {
+    return credentials;
+  };
+
+  public void setCredentials(Credentials credentials) {
+    this.credentials = credentials;
+  }
+
+
   @SuppressWarnings("rawtypes")
   public Map<String, List<AngularObject>> getAngularObjects() {
     return angularObjects;

http://git-wip-us.apache.org/repos/asf/incubator-zeppelin/blob/bdc78da0/zeppelin-zengine/src/main/java/org/apache/zeppelin/notebook/Notebook.java
----------------------------------------------------------------------
diff --git a/zeppelin-zengine/src/main/java/org/apache/zeppelin/notebook/Notebook.java b/zeppelin-zengine/src/main/java/org/apache/zeppelin/notebook/Notebook.java
index 986753c..4e61111 100644
--- a/zeppelin-zengine/src/main/java/org/apache/zeppelin/notebook/Notebook.java
+++ b/zeppelin-zengine/src/main/java/org/apache/zeppelin/notebook/Notebook.java
@@ -43,6 +43,7 @@ import org.apache.zeppelin.notebook.repo.NotebookRepoSync;
 import org.apache.zeppelin.resource.ResourcePoolUtils;
 import org.apache.zeppelin.scheduler.SchedulerFactory;
 import org.apache.zeppelin.search.SearchService;
+import org.apache.zeppelin.user.Credentials;
 import org.quartz.CronScheduleBuilder;
 import org.quartz.CronTrigger;
 import org.quartz.JobBuilder;
@@ -78,6 +79,7 @@ public class Notebook {
   private NotebookRepo notebookRepo;
   private SearchService notebookIndex;
   private NotebookAuthorization notebookAuthorization;
+  private Credentials credentials;
 
   /**
    * Main constructor \w manual Dependency Injection
@@ -97,7 +99,8 @@ public class Notebook {
       SchedulerFactory schedulerFactory,
       InterpreterFactory replFactory, JobListenerFactory jobListenerFactory,
       SearchService notebookIndex,
-      NotebookAuthorization notebookAuthorization) throws IOException, SchedulerException {
+      NotebookAuthorization notebookAuthorization,
+      Credentials credentials) throws IOException, SchedulerException {
     this.conf = conf;
     this.notebookRepo = notebookRepo;
     this.schedulerFactory = schedulerFactory;
@@ -105,6 +108,7 @@ public class Notebook {
     this.jobListenerFactory = jobListenerFactory;
     this.notebookIndex = notebookIndex;
     this.notebookAuthorization = notebookAuthorization;
+    this.credentials = credentials;
     quertzSchedFact = new org.quartz.impl.StdSchedulerFactory();
     quartzSched = quertzSchedFact.getScheduler();
     quartzSched.start();
@@ -146,7 +150,7 @@ public class Notebook {
    */
   public Note createNote(List<String> interpreterIds) throws IOException {
     NoteInterpreterLoader intpLoader = new NoteInterpreterLoader(replFactory);
-    Note note = new Note(notebookRepo, intpLoader, jobListenerFactory, notebookIndex);
+    Note note = new Note(notebookRepo, intpLoader, jobListenerFactory, notebookIndex, credentials);
     intpLoader.setNoteId(note.id());
     synchronized (notes) {
       notes.put(note.id(), note);
@@ -335,6 +339,7 @@ public class Notebook {
 
     //Manually inject ALL dependencies, as DI constructor was NOT used
     note.setIndex(this.notebookIndex);
+    note.setCredentials(this.credentials);
 
     NoteInterpreterLoader replLoader = new NoteInterpreterLoader(replFactory);
     note.setReplLoader(replLoader);

http://git-wip-us.apache.org/repos/asf/incubator-zeppelin/blob/bdc78da0/zeppelin-zengine/src/main/java/org/apache/zeppelin/notebook/NotebookAuthorization.java
----------------------------------------------------------------------
diff --git a/zeppelin-zengine/src/main/java/org/apache/zeppelin/notebook/NotebookAuthorization.java b/zeppelin-zengine/src/main/java/org/apache/zeppelin/notebook/NotebookAuthorization.java
index 212d608..82f6138 100644
--- a/zeppelin-zengine/src/main/java/org/apache/zeppelin/notebook/NotebookAuthorization.java
+++ b/zeppelin-zengine/src/main/java/org/apache/zeppelin/notebook/NotebookAuthorization.java
@@ -49,8 +49,7 @@ public class NotebookAuthorization {
     try {
       loadFromFile();
     } catch (IOException e) {
-      LOG.error("Error loading NotebookAuthorization");
-      e.printStackTrace();
+      LOG.error("Error loading NotebookAuthorization", e);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-zeppelin/blob/bdc78da0/zeppelin-zengine/src/main/java/org/apache/zeppelin/notebook/Paragraph.java
----------------------------------------------------------------------
diff --git a/zeppelin-zengine/src/main/java/org/apache/zeppelin/notebook/Paragraph.java b/zeppelin-zengine/src/main/java/org/apache/zeppelin/notebook/Paragraph.java
index 9a50a1a..36d466b 100644
--- a/zeppelin-zengine/src/main/java/org/apache/zeppelin/notebook/Paragraph.java
+++ b/zeppelin-zengine/src/main/java/org/apache/zeppelin/notebook/Paragraph.java
@@ -20,6 +20,9 @@ package org.apache.zeppelin.notebook;
 import org.apache.zeppelin.display.AngularObject;
 import org.apache.zeppelin.display.AngularObjectRegistry;
 import org.apache.zeppelin.user.AuthenticationInfo;
+import org.apache.zeppelin.user.Credentials;
+import org.apache.zeppelin.user.UserCredentials;
+import org.apache.zeppelin.user.UsernamePassword;
 import org.apache.zeppelin.display.GUI;
 import org.apache.zeppelin.display.Input;
 import org.apache.zeppelin.interpreter.*;
@@ -47,10 +50,11 @@ public class Paragraph extends Job implements Serializable, Cloneable {
 
   private transient NoteInterpreterLoader replLoader;
   private transient Note note;
+  private transient AuthenticationInfo authenticationInfo;
 
   String title;
   String text;
-  AuthenticationInfo authenticationInfo;
+  String user;
   Date dateUpdated;
   private Map<String, Object> config; // paragraph configs like isOpen, colWidth, etc
   public final GUI settings;          // form and parameter settings
@@ -70,6 +74,7 @@ public class Paragraph extends Job implements Serializable, Cloneable {
     title = null;
     text = null;
     authenticationInfo = null;
+    user = null;
     dateUpdated = null;
     settings = new GUI();
     config = new HashMap<String, Object>();
@@ -107,6 +112,7 @@ public class Paragraph extends Job implements Serializable, Cloneable {
 
   public void setAuthenticationInfo(AuthenticationInfo authenticationInfo) {
     this.authenticationInfo = authenticationInfo;
+    this.user = authenticationInfo.getUser();
   }
 
   public String getTitle() {
@@ -330,6 +336,14 @@ public class Paragraph extends Job implements Serializable, Cloneable {
     }
 
     final Paragraph self = this;
+
+    Credentials credentials = note.getCredentials();
+    if (authenticationInfo != null) {
+      UserCredentials userCredentials = credentials.getUserCredentials(
+              authenticationInfo.getUser());
+      authenticationInfo.setUserCredentials(userCredentials);
+    }
+
     InterpreterContext interpreterContext = new InterpreterContext(
             note.id(),
             getId(),

http://git-wip-us.apache.org/repos/asf/incubator-zeppelin/blob/bdc78da0/zeppelin-zengine/src/test/java/org/apache/zeppelin/notebook/NotebookTest.java
----------------------------------------------------------------------
diff --git a/zeppelin-zengine/src/test/java/org/apache/zeppelin/notebook/NotebookTest.java b/zeppelin-zengine/src/test/java/org/apache/zeppelin/notebook/NotebookTest.java
index 23a4b1b..80ef50f 100644
--- a/zeppelin-zengine/src/test/java/org/apache/zeppelin/notebook/NotebookTest.java
+++ b/zeppelin-zengine/src/test/java/org/apache/zeppelin/notebook/NotebookTest.java
@@ -46,6 +46,7 @@ import org.apache.zeppelin.scheduler.Job.Status;
 import org.apache.zeppelin.scheduler.JobListener;
 import org.apache.zeppelin.scheduler.SchedulerFactory;
 import org.apache.zeppelin.search.SearchService;
+import org.apache.zeppelin.user.Credentials;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
@@ -65,9 +66,12 @@ public class NotebookTest implements JobListenerFactory{
   private NotebookRepo notebookRepo;
   private InterpreterFactory factory;
   private DependencyResolver depResolver;
+  private NotebookAuthorization notebookAuthorization;
+  private Credentials credentials;
 
   @Before
   public void setUp() throws Exception {
+
     tmpDir = new File(System.getProperty("java.io.tmpdir")+"/ZeppelinLTest_"+System.currentTimeMillis());
     tmpDir.mkdirs();
     new File(tmpDir, "conf").mkdirs();
@@ -90,8 +94,12 @@ public class NotebookTest implements JobListenerFactory{
 
     SearchService search = mock(SearchService.class);
     notebookRepo = new VFSNotebookRepo(conf);
-    NotebookAuthorization notebookAuthorization = new NotebookAuthorization(conf);
-    notebook = new Notebook(conf, notebookRepo, schedulerFactory, factory, this, search, notebookAuthorization);
+    notebookAuthorization = new NotebookAuthorization(conf);
+    credentials = new Credentials(conf.credentialsPersist(), conf.getCredentialsPath());
+
+    notebook = new Notebook(conf, notebookRepo, schedulerFactory, factory, this, search,
+            notebookAuthorization, credentials);
+
   }
 
   @After
@@ -175,7 +183,7 @@ public class NotebookTest implements JobListenerFactory{
 
     Notebook notebook2 = new Notebook(
         conf, notebookRepo, schedulerFactory,
-        new InterpreterFactory(conf, null, null, null, depResolver), this, null, null);
+        new InterpreterFactory(conf, null, null, null, depResolver), this, null, null, null);
     assertEquals(1, notebook2.getAllNotes().size());
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-zeppelin/blob/bdc78da0/zeppelin-zengine/src/test/java/org/apache/zeppelin/notebook/repo/NotebookRepoSyncTest.java
----------------------------------------------------------------------
diff --git a/zeppelin-zengine/src/test/java/org/apache/zeppelin/notebook/repo/NotebookRepoSyncTest.java b/zeppelin-zengine/src/test/java/org/apache/zeppelin/notebook/repo/NotebookRepoSyncTest.java
index 69b50b4..1699d68 100644
--- a/zeppelin-zengine/src/test/java/org/apache/zeppelin/notebook/repo/NotebookRepoSyncTest.java
+++ b/zeppelin-zengine/src/test/java/org/apache/zeppelin/notebook/repo/NotebookRepoSyncTest.java
@@ -42,6 +42,7 @@ import org.apache.zeppelin.scheduler.JobListener;
 import org.apache.zeppelin.scheduler.SchedulerFactory;
 import org.apache.zeppelin.search.SearchService;
 import org.apache.zeppelin.search.LuceneSearch;
+import org.apache.zeppelin.user.Credentials;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
@@ -63,6 +64,7 @@ public class NotebookRepoSyncTest implements JobListenerFactory {
   private DependencyResolver depResolver;
   private SearchService search;
   private NotebookAuthorization notebookAuthorization;
+  private Credentials credentials;
   private static final Logger LOG = LoggerFactory.getLogger(NotebookRepoSyncTest.class);
   
   @Before
@@ -97,7 +99,9 @@ public class NotebookRepoSyncTest implements JobListenerFactory {
     search = mock(SearchService.class);
     notebookRepoSync = new NotebookRepoSync(conf);
     notebookAuthorization = new NotebookAuthorization(conf);
-    notebookSync = new Notebook(conf, notebookRepoSync, schedulerFactory, factory, this, search, notebookAuthorization);
+    credentials = new Credentials(conf.credentialsPersist(), conf.getCredentialsPath());
+    notebookSync = new Notebook(conf, notebookRepoSync, schedulerFactory, factory, this, search,
+            notebookAuthorization, credentials);
   }
 
   @After
@@ -222,7 +226,8 @@ public class NotebookRepoSyncTest implements JobListenerFactory {
     ZeppelinConfiguration vConf = ZeppelinConfiguration.create();
 
     NotebookRepoSync vRepoSync = new NotebookRepoSync(vConf);
-    Notebook vNotebookSync = new Notebook(vConf, vRepoSync, schedulerFactory, factory, this, search, null);
+    Notebook vNotebookSync = new Notebook(vConf, vRepoSync, schedulerFactory, factory, this, search,
+            notebookAuthorization, credentials);
 
     // one git versioned storage initialized
     assertThat(vRepoSync.getRepoCount()).isEqualTo(1);

http://git-wip-us.apache.org/repos/asf/incubator-zeppelin/blob/bdc78da0/zeppelin-zengine/src/test/java/org/apache/zeppelin/notebook/repo/VFSNotebookRepoTest.java
----------------------------------------------------------------------
diff --git a/zeppelin-zengine/src/test/java/org/apache/zeppelin/notebook/repo/VFSNotebookRepoTest.java b/zeppelin-zengine/src/test/java/org/apache/zeppelin/notebook/repo/VFSNotebookRepoTest.java
index 6058f82..e5915a9 100644
--- a/zeppelin-zengine/src/test/java/org/apache/zeppelin/notebook/repo/VFSNotebookRepoTest.java
+++ b/zeppelin-zengine/src/test/java/org/apache/zeppelin/notebook/repo/VFSNotebookRepoTest.java
@@ -80,7 +80,7 @@ public class VFSNotebookRepoTest implements JobListenerFactory {
 
     SearchService search = mock(SearchService.class);
     notebookRepo = new VFSNotebookRepo(conf);
-    notebook = new Notebook(conf, notebookRepo, schedulerFactory, factory, this, search, null);
+    notebook = new Notebook(conf, notebookRepo, schedulerFactory, factory, this, search, null, null);
   }
 
   @After

http://git-wip-us.apache.org/repos/asf/incubator-zeppelin/blob/bdc78da0/zeppelin-zengine/src/test/java/org/apache/zeppelin/search/LuceneSearchTest.java
----------------------------------------------------------------------
diff --git a/zeppelin-zengine/src/test/java/org/apache/zeppelin/search/LuceneSearchTest.java b/zeppelin-zengine/src/test/java/org/apache/zeppelin/search/LuceneSearchTest.java
index c744267..dcb68c8 100644
--- a/zeppelin-zengine/src/test/java/org/apache/zeppelin/search/LuceneSearchTest.java
+++ b/zeppelin-zengine/src/test/java/org/apache/zeppelin/search/LuceneSearchTest.java
@@ -286,7 +286,7 @@ public class LuceneSearchTest {
   }
 
   private Note newNote(String name) {
-    Note note = new Note(notebookRepoMock, replLoaderMock, null, notebookIndex);
+    Note note = new Note(notebookRepoMock, replLoaderMock, null, notebookIndex, null);
     note.setName(name);
     return note;
   }