You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@zeppelin.apache.org by fe...@apache.org on 2016/01/24 01:26:21 UTC

incubator-zeppelin git commit: ZEPPELIN-612 Introduce 'Configuration' page with WebSocket API / REST API

Repository: incubator-zeppelin
Updated Branches:
  refs/heads/master 5e89e75a7 -> 9308ee151


ZEPPELIN-612 Introduce 'Configuration' page with WebSocket API / REST API

### What is this PR for?

To introduce 'Configuration' page to Zeppelin GUI. Also introduce REST API / WebSocket operation.

### What type of PR is it?

Feature

### Todos

### Is there a relevant Jira issue?

https://issues.apache.org/jira/browse/ZEPPELIN-612

### How should this be tested?

- apply this PR
- build and start Zeppelin
- open UI and confirm Configuration tab is shown
- open Configuration page and confirm every configurations (properties) except password are shown

### Screenshots (if appropriate)

![configuration-screenshot](https://cloud.githubusercontent.com/assets/1317309/12372880/e679a282-bca9-11e5-97ee-8ab18e03a864.png)

### Questions:
* Does the licenses files need update? No
* Is there breaking changes for older versions? No
* Does this needs documentation? Yes, I've address it regarding REST API.

Author: Jungtaek Lim <ka...@gmail.com>

Closes #645 from HeartSaVioR/ZEPPELIN-612 and squashes the following commits:

8f584b8 [Jungtaek Lim] ZEPPELIN-612 addressed @felixcheung's review
14e8e7b [Jungtaek Lim] ZEPPELIN-612 replace ellipsis with scroll bar
051c8c5 [Jungtaek Lim] ZEPPELIN-612 cut too long value with ellipsis
aa61bb3 [Jungtaek Lim] ZEPPELIN-612 add missing license header
2a24c5c [Jungtaek Lim] ZEPPELIN-612 remove unused file
23fe68f [Jungtaek Lim] ZEPPELIN-612 Add 'Configuration' doc page to navigation
c341f82 [Jungtaek Lim] ZEPPELIN-612 Add 'Configuration' page
400de02 [Jungtaek Lim] ZEPPELIN-612 Introduce APIs which shows list of configurations via WebSocket / REST API


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

Branch: refs/heads/master
Commit: 9308ee151c4be6531455a8fcc9332832e8d369c8
Parents: 5e89e75
Author: Jungtaek Lim <ka...@gmail.com>
Authored: Thu Jan 21 23:48:07 2016 +0900
Committer: Felix Cheung <fe...@apache.org>
Committed: Sat Jan 23 16:26:20 2016 -0800

----------------------------------------------------------------------
 docs/_includes/themes/zeppelin/_navigation.html |   1 +
 docs/rest-api/rest-configuration.md             | 143 +++++++++++++++++++
 docs/rest-api/rest-interpreter.md               |   3 +-
 docs/rest-api/rest-notebook.md                  |   2 +-
 .../zeppelin/rest/ConfigurationsRestApi.java    |  81 +++++++++++
 .../apache/zeppelin/server/ZeppelinServer.java  |   8 +-
 .../org/apache/zeppelin/socket/Message.java     |   6 +-
 .../apache/zeppelin/socket/NotebookServer.java  |  19 +++
 .../rest/ConfigurationsRestApiTest.java         |  80 +++++++++++
 zeppelin-web/src/app/app.js                     |   4 +
 .../configuration/configuration.controller.js   |  36 +++++
 .../src/app/configuration/configuration.css     |  71 +++++++++
 .../src/app/configuration/configuration.html    |  55 +++++++
 zeppelin-web/src/components/navbar/navbar.html  |   3 +
 zeppelin-web/src/index.html                     |   2 +
 .../zeppelin/conf/ZeppelinConfiguration.java    |  40 ++++++
 16 files changed, 546 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-zeppelin/blob/9308ee15/docs/_includes/themes/zeppelin/_navigation.html
----------------------------------------------------------------------
diff --git a/docs/_includes/themes/zeppelin/_navigation.html b/docs/_includes/themes/zeppelin/_navigation.html
index 62db605..7481e32 100644
--- a/docs/_includes/themes/zeppelin/_navigation.html
+++ b/docs/_includes/themes/zeppelin/_navigation.html
@@ -76,6 +76,7 @@
                 <!-- li><span><b>REST API</b><span></li -->
                 <li><a href="{{BASE_PATH}}/rest-api/rest-interpreter.html">Interpreter API</a></li>
                 <li><a href="{{BASE_PATH}}/rest-api/rest-notebook.html">Notebook API</a></li>
+                <li><a href="{{BASE_PATH}}/rest-api/rest-configuration.html">Configuration API</a></li>
                 <li role="separator" class="divider"></li>
                 <!-- li><span><b>Development</b><span></li -->
                 <li><a href="{{BASE_PATH}}/development/writingzeppelininterpreter.html">Writing Zeppelin Interpreter</a></li>

http://git-wip-us.apache.org/repos/asf/incubator-zeppelin/blob/9308ee15/docs/rest-api/rest-configuration.md
----------------------------------------------------------------------
diff --git a/docs/rest-api/rest-configuration.md b/docs/rest-api/rest-configuration.md
new file mode 100644
index 0000000..587b065
--- /dev/null
+++ b/docs/rest-api/rest-configuration.md
@@ -0,0 +1,143 @@
+---
+layout: page
+title: "Configuration REST API"
+description: ""
+group: rest-api
+---
+<!--
+Licensed under the Apache License, Version 2.0 (the "License");
+you may not use this file except in compliance with the License.
+You may obtain a copy of the License at
+
+http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS,
+WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+See the License for the specific language governing permissions and
+limitations under the License.
+-->
+{% include JB/setup %}
+
+## Zeppelin REST API
+ Zeppelin provides several REST API's for interaction and remote activation of zeppelin functionality.
+ 
+ All REST API are available starting with the following endpoint ```http://[zeppelin-server]:[zeppelin-port]/api```
+ 
+ Note that zeppein REST API receive or return JSON objects, it it recommended you install some JSON viewers such as 
+ [JSONView](https://chrome.google.com/webstore/detail/jsonview/chklaanhfefbnpoihckbnefhakgolnmc)
+ 
+ 
+ If you work with zeppelin and find a need for an additional REST API please [file an issue or send us mail](../../community.html) 
+
+ <br />
+### Configuration REST API list
+  
+  <table class="table-configuration">
+    <col width="200">
+    <tr>
+      <th>List configurations</th>
+      <th></th>
+    </tr>
+    <tr>
+      <td>Description</td>
+      <td>This ```GET``` method return all key/value pair of configurations on the server.<br/> 
+      Note: For security reason, some pairs would not be shown.</td>
+    </tr>
+    <tr>
+      <td>URL</td>
+      <td>```http://[zeppelin-server]:[zeppelin-port]/api/configurations/all```</td>
+    </tr>
+    <tr>
+      <td>Success code</td>
+      <td>200</td>
+    </tr>
+    <tr>
+      <td> Fail code</td>
+      <td> 500 </td>
+    </tr>
+    <tr>
+      <td> sample JSON response
+      </td>
+      <td>
+        <pre>
+{  
+   "status":"OK",
+   "message":"",
+   "body":{  
+      "zeppelin.war.tempdir":"webapps",
+      "zeppelin.notebook.homescreen.hide":"false",
+      "zeppelin.interpreter.remoterunner":"bin/interpreter.sh",
+      "zeppelin.notebook.s3.user":"user",
+      "zeppelin.server.port":"8089",
+      "zeppelin.dep.localrepo":"local-repo",
+      "zeppelin.ssl.truststore.type":"JKS",
+      "zeppelin.ssl.keystore.path":"keystore",
+      "zeppelin.notebook.s3.bucket":"zeppelin",
+      "zeppelin.server.addr":"0.0.0.0",
+      "zeppelin.ssl.client.auth":"false",
+      "zeppelin.server.context.path":"/",
+      "zeppelin.ssl.keystore.type":"JKS",
+      "zeppelin.ssl.truststore.path":"truststore",
+      "zeppelin.interpreters":"org.apache.zeppelin.spark.SparkInterpreter,org.apache.zeppelin.spark.PySparkInterpreter,org.apache.zeppelin.spark.SparkSqlInterpreter,org.apache.zeppelin.spark.DepInterpreter,org.apache.zeppelin.markdown.Markdown,org.apache.zeppelin.angular.AngularInterpreter,org.apache.zeppelin.shell.ShellInterpreter,org.apache.zeppelin.hive.HiveInterpreter,org.apache.zeppelin.tajo.TajoInterpreter,org.apache.zeppelin.flink.FlinkInterpreter,org.apache.zeppelin.lens.LensInterpreter,org.apache.zeppelin.ignite.IgniteInterpreter,org.apache.zeppelin.ignite.IgniteSqlInterpreter,org.apache.zeppelin.cassandra.CassandraInterpreter,org.apache.zeppelin.geode.GeodeOqlInterpreter,org.apache.zeppelin.postgresql.PostgreSqlInterpreter,org.apache.zeppelin.phoenix.PhoenixInterpreter,org.apache.zeppelin.kylin.KylinInterpreter,org.apache.zeppelin.elasticsearch.ElasticsearchInterpreter,org.apache.zeppelin.scalding.ScaldingInterpreter",
+      "zeppelin.ssl":"false",
+      "zeppelin.notebook.autoInterpreterBinding":"true",
+      "zeppelin.notebook.homescreen":"",
+      "zeppelin.notebook.storage":"org.apache.zeppelin.notebook.repo.VFSNotebookRepo",
+      "zeppelin.interpreter.connect.timeout":"30000",
+      "zeppelin.anonymous.allowed":"true",
+      "zeppelin.server.allowed.origins":"*",
+      "zeppelin.encoding":"UTF-8"
+   }
+}
+        </pre>
+      </td>
+    </tr>
+  </table>
+  
+<br/>
+   
+  <table class="table-configuration">
+    <col width="200">
+    <tr>
+      <th>List configurations (prefix match)</th>
+      <th></th>
+    </tr>
+    <tr>
+      <td>Description</td>
+      <td>This ```GET``` method return all prefix matched key/value pair of configurations on the server.<br/> 
+      Note: For security reason, some pairs would not be shown.</td>
+    </tr>
+    <tr>
+      <td>URL</td>
+      <td>```http://[zeppelin-server]:[zeppelin-port]/api/configurations/prefix/[prefix]```</td>
+    </tr>
+    <tr>
+      <td>Success code</td>
+      <td>200</td>
+    </tr>
+    <tr>
+      <td> Fail code</td>
+      <td> 500 </td>
+    </tr>
+    <tr>
+      <td> sample JSON response
+      </td>
+      <td>
+        <pre>
+{  
+   "status":"OK",
+   "message":"",
+   "body":{  
+      "zeppelin.ssl.keystore.type":"JKS",
+      "zeppelin.ssl.truststore.path":"truststore",
+      "zeppelin.ssl.truststore.type":"JKS",
+      "zeppelin.ssl.keystore.path":"keystore",
+      "zeppelin.ssl":"false",
+      "zeppelin.ssl.client.auth":"false"
+   }
+}
+        </pre>
+      </td>
+    </tr>
+  </table>

http://git-wip-us.apache.org/repos/asf/incubator-zeppelin/blob/9308ee15/docs/rest-api/rest-interpreter.md
----------------------------------------------------------------------
diff --git a/docs/rest-api/rest-interpreter.md b/docs/rest-api/rest-interpreter.md
index ccdea3f..0db8470 100644
--- a/docs/rest-api/rest-interpreter.md
+++ b/docs/rest-api/rest-interpreter.md
@@ -23,10 +23,9 @@ limitations under the License.
  Zeppelin provides several REST API's for interaction and remote activation of zeppelin functionality.
  
  All REST API are available starting with the following endpoint `http://[zeppelin-server]:[zeppelin-port]/api`.
- Note that zeppein REST API receive or return JSON objects, it it recommended you install some JSON view such as 
+ Note that zeppein REST API receive or return JSON objects, it it recommended you install some JSON viewers such as 
  [JSON View](https://chrome.google.com/webstore/detail/jsonview/chklaanhfefbnpoihckbnefhakgolnmc).
  
- 
  If you work with zeppelin and find a need for an additional REST API, please [file an issue or send us mail](http://zeppelin.incubator.apache.org/community.html). 
 
  <br />

http://git-wip-us.apache.org/repos/asf/incubator-zeppelin/blob/9308ee15/docs/rest-api/rest-notebook.md
----------------------------------------------------------------------
diff --git a/docs/rest-api/rest-notebook.md b/docs/rest-api/rest-notebook.md
index a1e8de6..1f99657 100644
--- a/docs/rest-api/rest-notebook.md
+++ b/docs/rest-api/rest-notebook.md
@@ -24,7 +24,7 @@ limitations under the License.
  
  All REST APIs are available starting with the following endpoint ```http://[zeppelin-server]:[zeppelin-port]/api```
  
- Note that zeppelin REST APIs receive or return JSON objects, it is recommended for you to install some JSON viewer
+ Note that zeppelin REST APIs receive or return JSON objects, it is recommended for you to install some JSON viewers
   such as [JSONView](https://chrome.google.com/webstore/detail/jsonview/chklaanhfefbnpoihckbnefhakgolnmc)
  
  

http://git-wip-us.apache.org/repos/asf/incubator-zeppelin/blob/9308ee15/zeppelin-server/src/main/java/org/apache/zeppelin/rest/ConfigurationsRestApi.java
----------------------------------------------------------------------
diff --git a/zeppelin-server/src/main/java/org/apache/zeppelin/rest/ConfigurationsRestApi.java b/zeppelin-server/src/main/java/org/apache/zeppelin/rest/ConfigurationsRestApi.java
new file mode 100644
index 0000000..8e4ea48
--- /dev/null
+++ b/zeppelin-server/src/main/java/org/apache/zeppelin/rest/ConfigurationsRestApi.java
@@ -0,0 +1,81 @@
+/*
+ * 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 org.apache.zeppelin.conf.ZeppelinConfiguration;
+import org.apache.zeppelin.notebook.Notebook;
+import org.apache.zeppelin.server.JsonResponse;
+
+import javax.ws.rs.GET;
+import javax.ws.rs.Path;
+import javax.ws.rs.PathParam;
+import javax.ws.rs.Produces;
+import javax.ws.rs.core.Response;
+import javax.ws.rs.core.Response.Status;
+import java.util.Map;
+
+/**
+ * Configurations Rest API Endpoint
+ */
+@Path("/configurations")
+@Produces("application/json")
+public class ConfigurationsRestApi {
+
+  private Notebook notebook;
+
+  public ConfigurationsRestApi() {}
+
+  public ConfigurationsRestApi(Notebook notebook) {
+    this.notebook = notebook;
+  }
+
+  @GET
+  @Path("all")
+  public Response getAll() {
+    ZeppelinConfiguration conf = notebook.getConf();
+
+    Map<String, String> configurations = conf.dumpConfigurations(conf,
+        new ZeppelinConfiguration.ConfigurationKeyPredicate() {
+        @Override
+        public boolean apply(String key) {
+          return !key.contains("password");
+        }
+      }
+    );
+
+    return new JsonResponse(Status.OK, "", configurations).build();
+  }
+
+  @GET
+  @Path("prefix/{prefix}")
+  public Response getByPrefix(@PathParam("prefix") final String prefix) {
+    ZeppelinConfiguration conf = notebook.getConf();
+
+    Map<String, String> configurations = conf.dumpConfigurations(conf,
+        new ZeppelinConfiguration.ConfigurationKeyPredicate() {
+        @Override
+        public boolean apply(String key) {
+          return !key.contains("password") && key.startsWith(prefix);
+        }
+      }
+    );
+
+    return new JsonResponse(Status.OK, "", configurations).build();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-zeppelin/blob/9308ee15/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 dff75c7..760298f 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
@@ -35,10 +35,7 @@ import org.apache.zeppelin.interpreter.InterpreterFactory;
 import org.apache.zeppelin.notebook.Notebook;
 import org.apache.zeppelin.notebook.repo.NotebookRepo;
 import org.apache.zeppelin.notebook.repo.NotebookRepoSync;
-import org.apache.zeppelin.rest.InterpreterRestApi;
-import org.apache.zeppelin.rest.NotebookRestApi;
-import org.apache.zeppelin.rest.SecurityRestApi;
-import org.apache.zeppelin.rest.ZeppelinRestApi;
+import org.apache.zeppelin.rest.*;
 import org.apache.zeppelin.scheduler.SchedulerFactory;
 import org.apache.zeppelin.search.SearchService;
 import org.apache.zeppelin.search.LuceneSearch;
@@ -288,6 +285,9 @@ public class ZeppelinServer extends Application {
     SecurityRestApi securityApi = new SecurityRestApi();
     singletons.add(securityApi);
 
+    ConfigurationsRestApi settingsApi = new ConfigurationsRestApi(notebook);
+    singletons.add(settingsApi);
+
     return singletons;
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-zeppelin/blob/9308ee15/zeppelin-server/src/main/java/org/apache/zeppelin/socket/Message.java
----------------------------------------------------------------------
diff --git a/zeppelin-server/src/main/java/org/apache/zeppelin/socket/Message.java b/zeppelin-server/src/main/java/org/apache/zeppelin/socket/Message.java
index 4296e93..2a8e06d 100644
--- a/zeppelin-server/src/main/java/org/apache/zeppelin/socket/Message.java
+++ b/zeppelin-server/src/main/java/org/apache/zeppelin/socket/Message.java
@@ -100,7 +100,11 @@ public class Message {
     ANGULAR_OBJECT_UPDATE,  // [s-c] add/update angular object
     ANGULAR_OBJECT_REMOVE,  // [s-c] add angular object del
 
-    ANGULAR_OBJECT_UPDATED  // [c-s] angular object value updated
+    ANGULAR_OBJECT_UPDATED,  // [c-s] angular object value updated,
+
+    LIST_CONFIGURATIONS, // [c-s] ask all key/value pairs of configurations
+    CONFIGURATIONS_INFO // [s-c] all key/value pairs of configurations
+                  // @param settings serialized Map<String, String> object
   }
 
   public OP op;

http://git-wip-us.apache.org/repos/asf/incubator-zeppelin/blob/9308ee15/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 2c8a7c8..9a4a378 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
@@ -168,6 +168,9 @@ public class NotebookServer extends WebSocketServlet implements
           case ANGULAR_OBJECT_UPDATED:
             angularObjectUpdated(conn, notebook, messagereceived);
             break;
+          case LIST_CONFIGURATIONS:
+            sendAllConfigurations(conn, notebook);
+            break;
           default:
             broadcastNoteList();
             break;
@@ -717,6 +720,22 @@ public class NotebookServer extends WebSocketServlet implements
     }
   }
 
+  private void sendAllConfigurations(NotebookSocket conn, Notebook notebook)
+      throws IOException {
+    ZeppelinConfiguration conf = notebook.getConf();
+
+    Map<String, String> configurations = conf.dumpConfigurations(conf,
+        new ZeppelinConfiguration.ConfigurationKeyPredicate() {
+          @Override
+          public boolean apply(String key) {
+            return !key.contains("password");
+          }
+        });
+
+    conn.send(serializeMessage(new Message(OP.CONFIGURATIONS_INFO)
+        .put("configurations", configurations)));
+  }
+
   /**
    * This callback is for the paragraph that runs on ZeppelinServer
    * @param noteId

http://git-wip-us.apache.org/repos/asf/incubator-zeppelin/blob/9308ee15/zeppelin-server/src/test/java/org/apache/zeppelin/rest/ConfigurationsRestApiTest.java
----------------------------------------------------------------------
diff --git a/zeppelin-server/src/test/java/org/apache/zeppelin/rest/ConfigurationsRestApiTest.java b/zeppelin-server/src/test/java/org/apache/zeppelin/rest/ConfigurationsRestApiTest.java
new file mode 100644
index 0000000..1c1ebac
--- /dev/null
+++ b/zeppelin-server/src/test/java/org/apache/zeppelin/rest/ConfigurationsRestApiTest.java
@@ -0,0 +1,80 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.zeppelin.rest;
+
+import com.google.common.base.Predicate;
+import com.google.common.collect.Iterators;
+import com.google.gson.Gson;
+import com.google.gson.reflect.TypeToken;
+import org.apache.commons.httpclient.methods.GetMethod;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.util.Map;
+
+import static org.junit.Assert.assertTrue;
+
+public class ConfigurationsRestApiTest extends AbstractTestRestApi {
+  Gson gson = new Gson();
+
+  @BeforeClass
+  public static void init() throws Exception {
+    AbstractTestRestApi.startUp();
+  }
+
+  @AfterClass
+  public static void destroy() throws Exception {
+    AbstractTestRestApi.shutDown();
+  }
+
+  @Test
+  public void testGetAll() throws IOException {
+    GetMethod get = httpGet("/configurations/all");
+    Map<String, Object> resp = gson.fromJson(get.getResponseBodyAsString(),
+        new TypeToken<Map<String, Object>>(){}.getType());
+    Map<String, String> body = (Map<String, String>) resp.get("body");
+    assertTrue(body.size() > 0);
+    // it shouldn't have key/value pair which key contains "password"
+    assertTrue(Iterators.all(body.keySet().iterator(), new Predicate<String>() {
+        @Override
+        public boolean apply(String key) {
+          return !key.contains("password");
+        }
+      }
+    ));
+  }
+
+  @Test
+  public void testGetViaPrefix() throws IOException {
+    final String prefix = "zeppelin.server";
+    GetMethod get = httpGet("/configurations/prefix/" + prefix);
+    Map<String, Object> resp = gson.fromJson(get.getResponseBodyAsString(),
+        new TypeToken<Map<String, Object>>(){}.getType());
+    Map<String, String> body = (Map<String, String>) resp.get("body");
+    assertTrue(body.size() > 0);
+    assertTrue(Iterators.all(body.keySet().iterator(), new Predicate<String>() {
+          @Override
+          public boolean apply(String key) {
+            return !key.contains("password") && key.startsWith(prefix);
+          }
+        }
+    ));
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-zeppelin/blob/9308ee15/zeppelin-web/src/app/app.js
----------------------------------------------------------------------
diff --git a/zeppelin-web/src/app/app.js b/zeppelin-web/src/app/app.js
index 364ede7..d13faf0 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('/configuration', {
+                  templateUrl: 'app/configuration/configuration.html',
+                  controller: 'ConfigurationCtrl'
+                })
                 .when('/search/:searchTerm', {
                     templateUrl: 'app/search/result-list.html',
                     controller: 'SearchResultCtrl'

http://git-wip-us.apache.org/repos/asf/incubator-zeppelin/blob/9308ee15/zeppelin-web/src/app/configuration/configuration.controller.js
----------------------------------------------------------------------
diff --git a/zeppelin-web/src/app/configuration/configuration.controller.js b/zeppelin-web/src/app/configuration/configuration.controller.js
new file mode 100644
index 0000000..feecddf
--- /dev/null
+++ b/zeppelin-web/src/app/configuration/configuration.controller.js
@@ -0,0 +1,36 @@
+/*
+ * 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('ConfigurationCtrl', function($scope, $route, $routeParams, $location,
+                                                                          $rootScope, $http, baseUrlSrv) {
+  $scope.configrations = [];
+  $scope._ = _;
+
+  var getConfigurations = function() {
+    $http.get(baseUrlSrv.getRestApiBase()+'/configurations/all').
+    success(function(data, status, headers, config) {
+      $scope.configurations = data.body;
+    }).
+    error(function(data, status, headers, config) {
+      console.log('Error %o %o', status, data.message);
+    });
+  };
+
+  var init = function() {
+    getConfigurations();
+  };
+
+  init();
+});

http://git-wip-us.apache.org/repos/asf/incubator-zeppelin/blob/9308ee15/zeppelin-web/src/app/configuration/configuration.css
----------------------------------------------------------------------
diff --git a/zeppelin-web/src/app/configuration/configuration.css b/zeppelin-web/src/app/configuration/configuration.css
new file mode 100644
index 0000000..d11f6ec
--- /dev/null
+++ b/zeppelin-web/src/app/configuration/configuration.css
@@ -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.
+ */
+
+.configurationHead {
+  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;
+}
+
+.configurationHead .header {
+  font-family: 'Roboto', sans-serif;
+}
+
+.configuration .configuration-title {
+  font-size: 20px;
+  font-weight: bold;
+  color: #3071a9;
+  float: left;
+  margin-top: 0;
+}
+
+.configuration ul {
+  margin: 0;
+  padding: 0;
+}
+
+.configuration .configurationInfo {
+  list-style-type: none;
+}
+
+.configuration table {
+  table-layout: fixed;
+}
+
+.configuration table tr .configurationPropertyKey {
+  padding : 5px 5px 5px 5px;
+}
+
+.configuration table tr .configurationPropertyValue {
+  padding : 5px 5px 5px 5px;
+  display: block;
+  max-height: 100px;
+  overflow-y: auto;
+}
+
+.configuration h5 {
+  font-weight: bold;
+}
+
+.new_h3 {
+  margin-top: 1px;
+  padding-top: 7px;
+  float: left;
+}
+
+.hiding_overflow {
+  overflow:auto;
+}

http://git-wip-us.apache.org/repos/asf/incubator-zeppelin/blob/9308ee15/zeppelin-web/src/app/configuration/configuration.html
----------------------------------------------------------------------
diff --git a/zeppelin-web/src/app/configuration/configuration.html b/zeppelin-web/src/app/configuration/configuration.html
new file mode 100644
index 0000000..eded682
--- /dev/null
+++ b/zeppelin-web/src/app/configuration/configuration.html
@@ -0,0 +1,55 @@
+<!--
+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="configurationHead">
+  <div class="header">
+    <div class="row">
+      <div class="col-md-12">
+        <h3 class="new_h3">
+          Configurations
+        </h3>
+      </div>
+    </div>
+    <div class="row">
+      <div class="col-md-12">
+        Shows current configurations for Zeppelin Server.<br/>
+        Note: For security reasons, some key/value pairs including passwords would not be shown.
+      </div>
+    </div>
+  </div>
+</div>
+
+<div class="box width-full home">
+  <div>
+    <div class="row configuration">
+      <div class="col-md-12">
+        <table class="table table-striped">
+          <thead>
+          <tr>
+            <th style="width:30%">name</th>
+            <th>value</th>
+          </tr>
+          </thead>
+          <tr ng-repeat="(key, value) in configurations" >
+            <td>{{key}}</td>
+            <td>
+              <div class="hiding_overflow">
+                {{value}}
+              </div>
+            </td>
+          </tr>
+        </table>
+      </div>
+    </div>
+  </div>
+</div>

http://git-wip-us.apache.org/repos/asf/incubator-zeppelin/blob/9308ee15/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 86a8512..ca1c5fe 100644
--- a/zeppelin-web/src/components/navbar/navbar.html
+++ b/zeppelin-web/src/components/navbar/navbar.html
@@ -42,6 +42,9 @@ limitations under the License.
         <li>
           <a href="#/interpreter">Interpreter</a>
         </li>
+        <li>
+          <a href="#/configuration">Configuration</a>
+        </li>
       </ul>
 
 

http://git-wip-us.apache.org/repos/asf/incubator-zeppelin/blob/9308ee15/zeppelin-web/src/index.html
----------------------------------------------------------------------
diff --git a/zeppelin-web/src/index.html b/zeppelin-web/src/index.html
index 8a2c0f7..a8e082c 100644
--- a/zeppelin-web/src/index.html
+++ b/zeppelin-web/src/index.html
@@ -53,6 +53,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/configuration/configuration.css">
     <link rel="stylesheet" href="fonts/font-awesome.min.css">
     <link rel="stylesheet" href="fonts/simple-line-icons.css">
     <link rel="stylesheet" href="fonts/custom-font.css">
@@ -134,6 +135,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/configuration/configuration.controller.js"></script>
     <script src="app/notebook/paragraph/paragraph.controller.js"></script>
     <script src="app/search/result-list.controller.js"></script>
     <script src="components/arrayOrderingSrv/arrayOrdering.service.js"></script>

http://git-wip-us.apache.org/repos/asf/incubator-zeppelin/blob/9308ee15/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 5405d7b..231c5c6 100755
--- a/zeppelin-zengine/src/main/java/org/apache/zeppelin/conf/ZeppelinConfiguration.java
+++ b/zeppelin-zengine/src/main/java/org/apache/zeppelin/conf/ZeppelinConfiguration.java
@@ -19,7 +19,9 @@ package org.apache.zeppelin.conf;
 
 import java.net.URL;
 import java.util.Arrays;
+import java.util.HashMap;
 import java.util.List;
+import java.util.Map;
 
 import org.apache.commons.configuration.ConfigurationException;
 import org.apache.commons.configuration.XMLConfiguration;
@@ -373,6 +375,44 @@ public class ZeppelinConfiguration extends XMLConfiguration {
     return Arrays.asList(getString(ConfVars.ZEPPELIN_ALLOWED_ORIGINS).toLowerCase().split(","));
   }
 
+  public Map<String, String> dumpConfigurations(ZeppelinConfiguration conf,
+                                                ConfigurationKeyPredicate predicate) {
+    Map<String, String> configurations = new HashMap<>();
+
+    for (ZeppelinConfiguration.ConfVars v : ZeppelinConfiguration.ConfVars.values()) {
+      String key = v.getVarName();
+
+      if (!predicate.apply(key)) {
+        continue;
+      }
+
+      ConfVars.VarType type = v.getType();
+      Object value = null;
+      if (type == ConfVars.VarType.BOOLEAN) {
+        value = conf.getBoolean(v);
+      } else if (type == ConfVars.VarType.LONG) {
+        value = conf.getLong(v);
+      } else if (type == ConfVars.VarType.INT) {
+        value = conf.getInt(v);
+      } else if (type == ConfVars.VarType.FLOAT) {
+        value = conf.getFloat(v);
+      } else if (type == ConfVars.VarType.STRING) {
+        value = conf.getString(v);
+      }
+
+      if (value != null) {
+        configurations.put(key, value.toString());
+      }
+    }
+    return configurations;
+  }
+
+  /**
+   * Predication whether key/value pair should be included or not
+   */
+  public interface ConfigurationKeyPredicate {
+    boolean apply(String key);
+  }
 
   /**
    * Wrapper class.