You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@zeppelin.apache.org by mo...@apache.org on 2016/03/17 19:27:09 UTC

incubator-zeppelin git commit: ZEPPELIN-198 HDFS File Interpreter

Repository: incubator-zeppelin
Updated Branches:
  refs/heads/master 2fcc42b63 -> b45663d22


ZEPPELIN-198 HDFS File Interpreter

### What is this PR for?
This pull request is a follow of https://github.com/apache/incubator-zeppelin/pull/276 started by raj-bains.  The additional commits address comments from the pull request regarding string creation and error propagation for bad object requests.

### What type of PR is it?
[Bug Fix | Improvement | Feature | Documentation | Hot Fix | Refactoring]
Feature/Subtask
### Todos

### Is there a relevant Jira issue?
[ZEPPELIN-198](https://issues.apache.org/jira/browse/ZEPPELIN-198)
### How should this be tested?
Outline the steps to test the PR here.

### Screenshots (if appropriate)

![selection_001](https://cloud.githubusercontent.com/assets/9200575/13370345/87a922ac-dcd2-11e5-9217-66ec21c016de.png)
![selection_002](https://cloud.githubusercontent.com/assets/9200575/13370347/89a5603e-dcd2-11e5-9555-9c38167e8667.png)
![selection_003](https://cloud.githubusercontent.com/assets/9200575/13370359/b9bbadc8-dcd2-11e5-97bd-47d216f0c7da.png)
![selection_004](https://cloud.githubusercontent.com/assets/9200575/13370364/16ff9274-dcd3-11e5-9b4c-f324e7035c20.png)

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

Author: Tom Runyon <ru...@gmail.com>
Author: Raj Bains <ra...@Rajs-MacBook-Pro.local>

Closes #752 from runyontr/master and squashes the following commits:

f7bfef8 [Tom Runyon] ZEPPELIN-198 added transitive dependency lincense information
af16ce0 [Tom Runyon] ZEPPELIN-198 removed GPL2 reference from license file
5e9e131 [Tom Runyon] ZEPPELIN-198 Updated ZeppelinConfiguration to hold HDFS interpreter
9832622 [Tom Runyon] ZEPPELIN-198 Changed group for hdfs interpreter
c34a913 [Tom Runyon] ZEPPELIN-198 Updated licenses file to include org.glassfish.jersey.core
8d0ee3d [Tom Runyon] Merge https://github.com/apache/incubator-zeppelin
9f66514 [Tom Runyon] ZEPPELIN-198 Removed extra copy of configuration table and fixed formatting issues for hdfs.md
5938b0e [Tom Runyon] ZEPPELIN-198 Updated documentation to be consistent with naming between pom file and interpreter documentation.
67bbc5b [Tom Runyon] ZEPPELIN-198 Added navigation to hdfs interpreter
1c7a5c2 [Tom Runyon] ZEPPELIN-198 removed errant text in documentation.
56a5174 [Tom Runyon] ZEPPELIN-198 fixed logging to match standards
933c890 [Tom Runyon] MAINT Updated .gitignore file to remove zeppelin-server/derby.log
29540df [Tom Runyon] ZEPPELIN-198 removed zeppelin-server/derby.log
71d53d3 [Tom Runyon] ZEPPELIN-198 Changed pom name to Zeppelin File System Interpreters to match functionality
aec0512 [Tom Runyon] ZEPPELIN-198 Fixed compile error for error logging.
d24f4c0 [Tom Runyon] ZEPPELIN-198 Added error logging when returning error in interpet
227b815 [Tom Runyon] ZEPPELIN-198 Updated interpreter documentation.
b505391 [Tom Runyon] ZEPPELIN-198 Added completion functionality to HDFSInterpreter
32ed7cb [Tom Runyon] ZEPPELIN-198 Added -h flag for human readable byte sizes.  Updated string creation to StringBuilder.
797fd29 [Tom Runyon] Added org.glassfish.jersey.core to pom.xml file for hdfs intepretor
27e0438 [Tom Runyon] Modified string creation to use StringBuilder
79f0d90 [Tom Runyon] Merge branch 'master' of http://github.com/raj-bains/incubator-zeppelin
70507a8 [Raj Bains] Add Documentation and a missing dependency for HDFS File Browser
1239fe6 [Raj Bains] Merge remote-tracking branch 'upstream/master'
7d61e5f [Raj Bains] This is the first reviewed version of File Interpreter that adds basic ls, cd and pwd functionality against WebHDFS. It addresses ZEPPELIN-198
865e6ab [Raj Bains] Add File Interpreter, HDFS Interpreter and Tests


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

Branch: refs/heads/master
Commit: b45663d227c222d3d5c1e9b64fac5df48509bd1a
Parents: 2fcc42b
Author: Tom Runyon <ru...@gmail.com>
Authored: Sun Mar 13 22:13:00 2016 -0400
Committer: Lee moon soo <mo...@apache.org>
Committed: Thu Mar 17 11:27:08 2016 -0700

----------------------------------------------------------------------
 .gitignore                                      |   1 +
 conf/zeppelin-site.xml.template                 |   2 +-
 docs/_includes/themes/zeppelin/_navigation.html |   1 +
 docs/interpreter/hdfs.md                        |  56 ++++
 file/pom.xml                                    | 146 ++++++++
 .../apache/zeppelin/file/FileInterpreter.java   | 171 ++++++++++
 .../org/apache/zeppelin/file/HDFSCommand.java   | 156 +++++++++
 .../zeppelin/file/HDFSFileInterpreter.java      | 330 +++++++++++++++++++
 .../zeppelin/file/HDFSFileInterpreterTest.java  | 209 ++++++++++++
 pom.xml                                         |   1 +
 zeppelin-distribution/src/bin_license/LICENSE   |   4 +
 .../zeppelin/conf/ZeppelinConfiguration.java    |   1 +
 12 files changed, 1077 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-zeppelin/blob/b45663d2/.gitignore
----------------------------------------------------------------------
diff --git a/.gitignore b/.gitignore
index b7cec24..3f3d654 100644
--- a/.gitignore
+++ b/.gitignore
@@ -12,6 +12,7 @@
 spark/derby.log
 spark/metastore_db
 spark-1.*-bin-hadoop*
+zeppelin-server/derby.log
 
 lens/lens-cli-hist.log
 

http://git-wip-us.apache.org/repos/asf/incubator-zeppelin/blob/b45663d2/conf/zeppelin-site.xml.template
----------------------------------------------------------------------
diff --git a/conf/zeppelin-site.xml.template b/conf/zeppelin-site.xml.template
index 16e26b7..730de7b 100755
--- a/conf/zeppelin-site.xml.template
+++ b/conf/zeppelin-site.xml.template
@@ -138,7 +138,7 @@
 
 <property>
   <name>zeppelin.interpreters</name>
-  <value>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.jdbc.JDBCInterpreter,org.apache.zeppelin.phoenix.PhoenixInterpreter,org.apache.zeppelin.kylin.KylinInterpreter,org.apache.zeppelin.elasticsearch.ElasticsearchInterpreter,org.apache.zeppelin.scalding.ScaldingInterpreter,org.apache.zeppelin.alluxio.AlluxioIn
 terpreter,org.apache.zeppelin.hbase.HbaseInterpreter</value>
+  <value>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.file.HDFSFileInterpreter,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.jdbc.JDBCInterpreter,org.apache.zeppelin.phoenix.PhoenixInterpreter,org.apache.zeppelin.kylin.KylinInterpreter,org.apache.zeppelin.elasticsearch.ElasticsearchInterpreter,org.apache.zeppelin.scalding.ScaldingInte
 rpreter,org.apache.zeppelin.alluxio.AlluxioInterpreter,org.apache.zeppelin.hbase.HbaseInterpreter</value>
   <description>Comma separated interpreter configurations. First interpreter become a default</description>
 </property>
 

http://git-wip-us.apache.org/repos/asf/incubator-zeppelin/blob/b45663d2/docs/_includes/themes/zeppelin/_navigation.html
----------------------------------------------------------------------
diff --git a/docs/_includes/themes/zeppelin/_navigation.html b/docs/_includes/themes/zeppelin/_navigation.html
index 994e5db..fb09a9e 100644
--- a/docs/_includes/themes/zeppelin/_navigation.html
+++ b/docs/_includes/themes/zeppelin/_navigation.html
@@ -46,6 +46,7 @@
                 <li><a href="{{BASE_PATH}}/interpreter/flink.html">Flink</a></li>
                 <li><a href="{{BASE_PATH}}/interpreter/geode.html">Geode</a></li>
                 <li><a href="{{BASE_PATH}}/interpreter/hbase.html">HBase</a></li>
+                <li><a href="{{BASE_PATH}}/interpreter/hdfs.html">HDFS</a></li>
                 <li><a href="{{BASE_PATH}}/interpreter/hive.html">Hive</a></li>
                 <li><a href="{{BASE_PATH}}/interpreter/ignite.html">Ignite</a></li>
                 <li><a href="{{BASE_PATH}}/interpreter/jdbc.html">JDBC</a></li>

http://git-wip-us.apache.org/repos/asf/incubator-zeppelin/blob/b45663d2/docs/interpreter/hdfs.md
----------------------------------------------------------------------
diff --git a/docs/interpreter/hdfs.md b/docs/interpreter/hdfs.md
new file mode 100644
index 0000000..f29755f
--- /dev/null
+++ b/docs/interpreter/hdfs.md
@@ -0,0 +1,56 @@
+---
+layout: page
+title: "HDFS File System Interpreter"
+description: ""
+group: manual
+---
+{% include JB/setup %}
+
+## HDFS File System Interpreter for Apache Zeppelin
+
+[Hadoop File System](http://hadoop.apache.org/) is a distributed, fault tolerant file system part of the hadoop project and is often used as storage for distributed processing engines like [Hadoop MapReduce](http://hadoop.apache.org/) and [Apache Spark](http://spark.apache.org/) or underlying file systems like [Alluxio](http://www.alluxio.org/).
+
+## Configuration
+<table class="table-configuration">
+  <tr>
+    <th>Property</th>
+    <th>Default</th>
+    <th>Description</th>
+  </tr>
+  <tr>
+    <td>hdfs.url</td>
+    <td>http://localhost:50070/webhdfs/v1/</td>
+    <td>The URL for WebHDFS</td>
+  </tr>
+  <tr>
+    <td>hdfs.user</td>
+    <td>hdfs</td>
+    <td>The WebHDFS user</td>
+  </tr>
+  <tr>
+    <td>hdfs.maxlength</td>
+    <td>1000</td>
+    <td>Maximum number of lines of results fetched</td>
+  </tr>
+</table>
+
+<br/>
+This interpreter connects to HDFS using the HTTP WebHDFS interface.
+It supports the basic shell file commands applied to HDFS, it currently only supports browsing.
+
+* You can use <i>ls [PATH]</i> and <i>ls -l [PATH]</i> to list a directory. If the path is missing, then the current directory is listed.  <i>ls </i> supports a <i>-h</i> flag for human readable file sizes.
+* You can use <i>cd [PATH]</i> to change your current directory by giving a relative or an absolute path.
+* You can invoke <i>pwd</i> to see your current directory.
+
+> **Tip :** Use ( Ctrl + . ) for autocompletion.
+
+### Create Interpreter 
+
+In a notebook, to enable the **HDFS** interpreter, click the **Gear** icon and select **HDFS**.
+
+
+#### WebHDFS REST API
+You can confirm that you're able to access the WebHDFS API by running a curl command against the WebHDFS end point provided to the interpreter.
+
+Here is an example:
+$> curl "http://localhost:50070/webhdfs/v1/?op=LISTSTATUS"
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-zeppelin/blob/b45663d2/file/pom.xml
----------------------------------------------------------------------
diff --git a/file/pom.xml b/file/pom.xml
new file mode 100644
index 0000000..586f41a
--- /dev/null
+++ b/file/pom.xml
@@ -0,0 +1,146 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  ~ Licensed to the Apache Software Foundation (ASF) under one or more
+  ~ contributor license agreements.  See the NOTICE file distributed with
+  ~ this work for additional information regarding copyright ownership.
+  ~ The ASF licenses this file to You under the Apache License, Version 2.0
+  ~ (the "License"); you may not use this file except in compliance with
+  ~ the License.  You may obtain a copy of the License at
+  ~
+  ~    http://www.apache.org/licenses/LICENSE-2.0
+  ~
+  ~ Unless required by applicable law or agreed to in writing, software
+  ~ distributed under the License is distributed on an "AS IS" BASIS,
+  ~ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  ~ See the License for the specific language governing permissions and
+  ~ limitations under the License.
+  -->
+
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
+  <modelVersion>4.0.0</modelVersion>
+
+  <parent>
+    <artifactId>zeppelin</artifactId>
+    <groupId>org.apache.zeppelin</groupId>
+    <version>0.6.0-incubating-SNAPSHOT</version>
+  </parent>
+
+  <groupId>org.apache.zeppelin</groupId>
+  <artifactId>zeppelin-file</artifactId>
+  <packaging>jar</packaging>
+  <version>0.6.0-incubating-SNAPSHOT</version>
+  <name>Zeppelin File System Interpreters</name>
+  <url>http://www.apache.org</url>
+
+  <dependencies>
+    <dependency>
+      <groupId>org.apache.zeppelin</groupId>
+      <artifactId>zeppelin-interpreter</artifactId>
+      <version>${project.version}</version>
+      <scope>provided</scope>
+    </dependency>
+
+    <dependency>
+      <groupId>javax.ws.rs</groupId>
+      <artifactId>javax.ws.rs-api</artifactId>
+      <version>2.0</version>
+    </dependency>
+
+    <dependency>
+      <groupId>org.slf4j</groupId>
+      <artifactId>slf4j-api</artifactId>
+    </dependency>
+
+    <dependency>
+      <groupId>org.slf4j</groupId>
+      <artifactId>slf4j-log4j12</artifactId>
+    </dependency>
+
+    <dependency>
+      <groupId>org.glassfish.jersey.core</groupId>
+      <artifactId>jersey-common</artifactId>
+      <version>2.22.2</version>
+    </dependency>
+
+
+    <dependency>
+      <groupId>junit</groupId>
+      <artifactId>junit</artifactId>
+      <scope>test</scope>
+    </dependency>
+  </dependencies>
+
+  <build>
+    <plugins>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-deploy-plugin</artifactId>
+        <version>2.7</version>
+        <configuration>
+          <skip>true</skip>
+        </configuration>
+      </plugin>
+
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-surefire-plugin</artifactId>
+        <version>2.18.1</version>
+      </plugin>
+
+      <plugin>
+        <artifactId>maven-enforcer-plugin</artifactId>
+        <version>1.3.1</version>            
+        <executions> 
+          <execution> 
+            <id>enforce</id> 
+            <phase>none</phase> 
+          </execution>
+        </executions>
+      </plugin>
+
+      <plugin>
+        <artifactId>maven-dependency-plugin</artifactId>
+        <version>2.8</version>
+        <executions>
+          <execution>
+            <id>copy-dependencies</id>
+            <phase>package</phase>
+            <goals>
+              <goal>copy-dependencies</goal>
+            </goals>
+            <configuration>
+              <outputDirectory>${project.build.directory}/../../interpreter/file</outputDirectory>
+              <overWriteReleases>false</overWriteReleases>
+              <overWriteSnapshots>false</overWriteSnapshots>
+              <overWriteIfNewer>true</overWriteIfNewer>
+              <includeScope>runtime</includeScope>
+            </configuration>
+          </execution>
+          <execution>
+            <id>copy-artifact</id>
+            <phase>package</phase>
+            <goals>
+              <goal>copy</goal>
+            </goals>
+            <configuration>
+              <outputDirectory>${project.build.directory}/../../interpreter/file</outputDirectory>
+              <overWriteReleases>false</overWriteReleases>
+              <overWriteSnapshots>false</overWriteSnapshots>
+              <overWriteIfNewer>true</overWriteIfNewer>
+              <!--<includeScope>runtime</includeScope>-->
+              <artifactItems>
+                <artifactItem>
+                  <groupId>${project.groupId}</groupId>
+                  <artifactId>${project.artifactId}</artifactId>
+                  <version>${project.version}</version>
+                  <type>${project.packaging}</type>
+                </artifactItem>
+              </artifactItems>              
+            </configuration>
+          </execution>
+        </executions>
+      </plugin>
+    </plugins>
+  </build>
+
+</project>

http://git-wip-us.apache.org/repos/asf/incubator-zeppelin/blob/b45663d2/file/src/main/java/org/apache/zeppelin/file/FileInterpreter.java
----------------------------------------------------------------------
diff --git a/file/src/main/java/org/apache/zeppelin/file/FileInterpreter.java b/file/src/main/java/org/apache/zeppelin/file/FileInterpreter.java
new file mode 100644
index 0000000..4d50ce5
--- /dev/null
+++ b/file/src/main/java/org/apache/zeppelin/file/FileInterpreter.java
@@ -0,0 +1,171 @@
+/**
+ * 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.file;
+
+import org.apache.zeppelin.interpreter.Interpreter;
+import org.apache.zeppelin.interpreter.InterpreterContext;
+import org.apache.zeppelin.interpreter.InterpreterResult;
+import org.apache.zeppelin.interpreter.InterpreterResult.Code;
+import org.apache.zeppelin.interpreter.InterpreterResult.Type;
+import org.apache.zeppelin.scheduler.Scheduler;
+import org.apache.zeppelin.scheduler.SchedulerFactory;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.*;
+
+/**
+ * File interpreter for Zeppelin.
+ *
+ */
+public abstract class FileInterpreter extends Interpreter {
+  Logger logger = LoggerFactory.getLogger(FileInterpreter.class);
+  String currentDir = null;
+  CommandArgs args = null;
+
+  public FileInterpreter(Properties property) {
+    super(property);
+    currentDir = new String("/");
+  }
+
+  /**
+   * Handling the arguments of the command
+   */
+  public class CommandArgs {
+    public String input = null;
+    public String command = null;
+    public ArrayList<String> args = null;
+    public HashSet<Character> flags = null;
+
+    public CommandArgs(String cmd) {
+      input = cmd;
+      args = new ArrayList();
+      flags = new HashSet();
+    }
+
+    private void parseArg(String arg) {
+      if (arg.charAt(0) == '-') {                   // handle flags
+        for (int i = 0; i < arg.length(); i++) {
+          Character c = arg.charAt(i);
+          flags.add(c);
+        }
+      } else {                                      // handle other args
+        args.add(arg);
+      }
+    }
+
+    public void parseArgs() {
+      if (input == null)
+        return;
+      StringTokenizer st = new StringTokenizer(input);
+      if (st.hasMoreTokens()) {
+        command = st.nextToken();
+        while (st.hasMoreTokens())
+          parseArg(st.nextToken());
+      }
+    }
+  }
+
+  // Functions that each file system implementation must override
+
+  public abstract String listAll(String path);
+
+  public abstract boolean isDirectory(String path);
+
+  // Combine paths, takes care of arguments such as ..
+
+  protected String getNewPath(String argument){
+    Path arg = Paths.get(argument);
+    Path ret = arg.isAbsolute() ? arg : Paths.get(currentDir, argument);
+    return ret.normalize().toString();
+  }
+
+  // Handle the command handling uniformly across all file systems
+
+  @Override
+  public InterpreterResult interpret(String cmd, InterpreterContext contextInterpreter) {
+    logger.info("Run File command '" + cmd + "'");
+
+    args = new CommandArgs(cmd);
+    args.parseArgs();
+
+    if (args.command == null) {
+      logger.info("Error: No command");
+      return new InterpreterResult(Code.ERROR, Type.TEXT, "No command");
+    }
+
+    // Simple parsing of the command
+
+    if (args.command.equals("cd")) {
+
+      String newPath = !args.args.isEmpty() ? getNewPath(args.args.get(0)) : currentDir;
+      if (!isDirectory(newPath))
+        return new InterpreterResult(Code.ERROR, Type.TEXT, newPath + ": No such directory");
+
+      currentDir = newPath;
+      return new InterpreterResult(Code.SUCCESS, Type.TEXT, "OK");
+
+    } else if (args.command.equals("ls")) {
+
+      String newPath = !args.args.isEmpty() ? getNewPath(args.args.get(0)) : currentDir;
+      try {
+        String results = listAll(newPath);
+        return new InterpreterResult(Code.SUCCESS, Type.TEXT, results);
+      } catch (Exception e) {
+        logger.error("Error listing files in path " + newPath, e);
+        return new InterpreterResult(Code.ERROR, Type.TEXT, e.getMessage());
+      }
+
+    } else if (args.command.equals("pwd")) {
+
+      return new InterpreterResult(Code.SUCCESS, Type.TEXT, currentDir);
+
+    } else {
+
+      return new InterpreterResult(Code.ERROR, Type.TEXT, "Unknown command");
+
+    }
+  }
+
+  @Override
+  public void cancel(InterpreterContext context) {
+  }
+
+  @Override
+  public FormType getFormType() {
+    return FormType.SIMPLE;
+  }
+
+  @Override
+  public int getProgress(InterpreterContext context) {
+    return 0;
+  }
+
+  @Override
+  public Scheduler getScheduler() {
+    return SchedulerFactory.singleton().createOrGetFIFOScheduler(
+        FileInterpreter.class.getName() + this.hashCode());
+  }
+
+  @Override
+  public List<String> completion(String buf, int cursor) {
+    return null;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-zeppelin/blob/b45663d2/file/src/main/java/org/apache/zeppelin/file/HDFSCommand.java
----------------------------------------------------------------------
diff --git a/file/src/main/java/org/apache/zeppelin/file/HDFSCommand.java b/file/src/main/java/org/apache/zeppelin/file/HDFSCommand.java
new file mode 100644
index 0000000..94508cd
--- /dev/null
+++ b/file/src/main/java/org/apache/zeppelin/file/HDFSCommand.java
@@ -0,0 +1,156 @@
+/**
+ * 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.file;
+
+import java.net.URL;
+import java.net.HttpURLConnection;
+import java.io.BufferedReader;
+import java.io.InputStreamReader;
+import javax.ws.rs.core.UriBuilder;
+import org.slf4j.Logger;
+
+/**
+ * Definition and HTTP invocation methods for all WebHDFS commands
+ *
+ */
+public class HDFSCommand {
+
+  /**
+   * Type of HTTP request
+   */
+  public enum HttpType {
+    GET,
+    PUT
+  }
+
+  /**
+   * Definition of WebHDFS operator
+   */
+  public class Op {
+    public String op;
+    public HttpType cmd;
+    public int minArgs;
+
+    public Op(String op, HttpType cmd, int minArgs) {
+      this.op = op;
+      this.cmd = cmd;
+      this.minArgs = minArgs;
+    }
+  }
+
+  /**
+   * Definition of argument to an operator
+   */
+  public class Arg {
+    public String key;
+    public String value;
+
+    public Arg(String key, String value) {
+      this.key = key;
+      this.value = value;
+    }
+  }
+
+  // How to connect to WebHDFS
+  String url = null;
+  String user = null;
+  int maxLength = 0;
+  Logger logger;
+
+  // Define all the commands available
+  public Op getFileStatus = new Op("GETFILESTATUS", HttpType.GET, 0);
+  public Op listStatus = new Op("LISTSTATUS", HttpType.GET, 0);
+
+  public HDFSCommand(String url, String user, Logger logger, int maxLength) {
+    super();
+    this.url = url;
+    this.user = user;
+    this.maxLength = maxLength;
+    this.logger = logger;
+  }
+
+  public String checkArgs(Op op, String path, Arg[] args) throws Exception {
+    if (op == null ||
+        path == null ||
+        (op.minArgs > 0 &&
+            (args == null ||
+                args.length != op.minArgs)))
+    {
+      String a = "";
+      a = (op != null) ? a + op.op + "\n" : a;
+      a = (path != null) ? a + path + "\n" : a;
+      a = (args != null) ? a + args + "\n" : a;
+      return a;
+    }
+    return null;
+  }
+
+
+  // The operator that runs all commands
+  public String runCommand(Op op, String path, Arg[] args) throws Exception {
+
+    // Check arguments
+    String error = checkArgs(op, path, args);
+    if (error != null) {
+      logger.error("Bad arguments to command: " + error);
+      return "ERROR: BAD ARGS";
+    }
+
+    // Build URI
+    UriBuilder builder = UriBuilder
+        .fromPath(url)
+        .path(path)
+        .queryParam("op", op.op);
+
+    if (args != null) {
+      for (Arg a : args) {
+        builder = builder.queryParam(a.key, a.value);
+      }
+    }
+    java.net.URI uri = builder.build();
+
+    // Connect and get response string
+    URL hdfsUrl = uri.toURL();
+    HttpURLConnection con = (HttpURLConnection) hdfsUrl.openConnection();
+
+    if (op.cmd == HttpType.GET) {
+      con.setRequestMethod("GET");
+      int responseCode = con.getResponseCode();
+      logger.info("Sending 'GET' request to URL : " + hdfsUrl);
+      logger.info("Response Code : " + responseCode);
+
+      BufferedReader in = new BufferedReader(
+          new InputStreamReader(con.getInputStream()));
+      String inputLine;
+      StringBuffer response = new StringBuffer();
+
+      int i = 0;
+      while ((inputLine = in.readLine()) != null) {
+        if (inputLine.length() < maxLength)
+          response.append(inputLine);
+        i++;
+        if (i >= maxLength)
+          break;
+      }
+      in.close();
+      return response.toString();
+    }
+    return null;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-zeppelin/blob/b45663d2/file/src/main/java/org/apache/zeppelin/file/HDFSFileInterpreter.java
----------------------------------------------------------------------
diff --git a/file/src/main/java/org/apache/zeppelin/file/HDFSFileInterpreter.java b/file/src/main/java/org/apache/zeppelin/file/HDFSFileInterpreter.java
new file mode 100644
index 0000000..245093e
--- /dev/null
+++ b/file/src/main/java/org/apache/zeppelin/file/HDFSFileInterpreter.java
@@ -0,0 +1,330 @@
+/**
+ * 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.file;
+
+import java.text.SimpleDateFormat;
+import java.util.*;
+
+import com.google.gson.Gson;
+import org.apache.commons.lang.StringUtils;
+import org.apache.zeppelin.interpreter.Interpreter;
+import org.apache.zeppelin.interpreter.InterpreterException;
+import org.apache.zeppelin.interpreter.InterpreterPropertyBuilder;
+
+/**
+ * HDFS implementation of File interpreter for Zeppelin.
+ *
+ */
+public class HDFSFileInterpreter extends FileInterpreter {
+  static final String HDFS_URL = "hdfs.url";
+  static final String HDFS_USER = "hdfs.user";
+  static final String HDFS_MAXLENGTH = "hdfs.maxlength";
+
+  static {
+    Interpreter.register(
+        "hdfs",
+        "file",
+        HDFSFileInterpreter.class.getName(),
+        new InterpreterPropertyBuilder()
+            .add(HDFS_URL, "http://localhost:50070/webhdfs/v1/", "The URL for WebHDFS")
+            .add(HDFS_USER, "hdfs", "The WebHDFS user")
+            .add(HDFS_MAXLENGTH, "1000", "Maximum number of lines of results fetched").build());
+  }
+
+  Exception exceptionOnConnect = null;
+  HDFSCommand cmd = null;
+  Gson gson = null;
+
+  public void prepare() {
+    String userName = getProperty(HDFS_USER);
+    String hdfsUrl = getProperty(HDFS_URL);
+    int i = Integer.parseInt(getProperty(HDFS_MAXLENGTH));
+    cmd = new HDFSCommand(hdfsUrl, userName, logger, i);
+    gson = new Gson();
+  }
+
+  public HDFSFileInterpreter(Properties property){
+    super(property);
+    prepare();
+  }
+
+  /**
+   * Status of one file
+   *
+   * matches returned JSON
+   */
+  public class OneFileStatus {
+    public long accessTime;
+    public int blockSize;
+    public int childrenNum;
+    public int fileId;
+    public String group;
+    public long length;
+    public long modificationTime;
+    public String owner;
+    public String pathSuffix;
+    public String permission;
+    public int replication;
+    public int storagePolicy;
+    public String type;
+    public String toString() {
+      StringBuilder sb = new StringBuilder();
+      sb.append("\nAccessTime = " + accessTime);
+      sb.append("\nBlockSize = " + blockSize);
+      sb.append("\nChildrenNum = " + childrenNum);
+      sb.append("\nFileId = " + fileId);
+      sb.append("\nGroup = " + group);
+      sb.append("\nLength = " + length);
+      sb.append("\nModificationTime = " + modificationTime);
+      sb.append("\nOwner = " + owner);
+      sb.append("\nPathSuffix = " + pathSuffix);
+      sb.append("\nPermission = " + permission);
+      sb.append("\nReplication = " + replication);
+      sb.append("\nStoragePolicy = " + storagePolicy);
+      sb.append("\nType = " + type);
+      return sb.toString();
+    }
+  }
+
+  /**
+   * Status of one file
+   *
+   * matches returned JSON
+   */
+  public class SingleFileStatus {
+    public OneFileStatus FileStatus;
+  }
+
+  /**
+   * Status of all files in a directory
+   *
+   * matches returned JSON
+   */
+  public class MultiFileStatus {
+    public OneFileStatus[] FileStatus;
+  }
+
+  /**
+   * Status of all files in a directory
+   *
+   * matches returned JSON
+   */
+  public class AllFileStatus {
+    public MultiFileStatus FileStatuses;
+  }
+
+  // tests whether we're able to connect to HDFS
+
+  private void testConnection() {
+    try {
+      if (isDirectory("/"))
+        logger.info("Successfully created WebHDFS connection");
+    } catch (Exception e) {
+      logger.error("testConnection: Cannot open WebHDFS connection. Bad URL: " + "/", e);
+      exceptionOnConnect = e;
+    }
+  }
+
+  @Override
+  public void open() {
+    testConnection();
+  }
+
+  @Override
+  public void close() {
+  }
+
+  private String listDir(String path) throws Exception {
+    return cmd.runCommand(cmd.listStatus, path, null);
+  }
+
+  private String listPermission(OneFileStatus fs){
+    StringBuilder sb = new StringBuilder();
+    sb.append(fs.type.equalsIgnoreCase("Directory") ? 'd' : '-');
+    int p = Integer.parseInt(fs.permission, 16);
+    sb.append(((p & 0x400) == 0) ? '-' : 'r');
+    sb.append(((p & 0x200) == 0) ? '-' : 'w');
+    sb.append(((p & 0x100) == 0) ? '-' : 'x');
+    sb.append(((p & 0x40)  == 0) ? '-' : 'r');
+    sb.append(((p & 0x20)  == 0) ? '-' : 'w');
+    sb.append(((p & 0x10)  == 0) ? '-' : 'x');
+    sb.append(((p & 0x4)   == 0) ? '-' : 'r');
+    sb.append(((p & 0x2)   == 0) ? '-' : 'w');
+    sb.append(((p & 0x1)   == 0) ? '-' : 'x');
+    return sb.toString();
+  }
+  private String listDate(OneFileStatus fs) {
+    return new SimpleDateFormat("yyyy-MM-dd HH:mm").format(new Date(fs.modificationTime));
+  }
+  private String ListOne(String path, OneFileStatus fs) {
+    if (args.flags.contains(new Character('l'))) {
+      StringBuilder sb = new StringBuilder();
+      sb.append(listPermission(fs) + "\t");
+      sb.append(((fs.replication == 0) ? "-" : fs.replication) + "\t ");
+      sb.append(fs.owner + "\t");
+      sb.append(fs.group + "\t");
+      if (args.flags.contains(new Character('h'))){ //human readable
+        sb.append(humanReadableByteCount(fs.length) + "\t\t");
+      } else {
+        sb.append(fs.length + "\t");
+      }
+      sb.append(listDate(fs) + "GMT\t");
+      sb.append((path.length() == 1) ? path + fs.pathSuffix : path + '/' + fs.pathSuffix);
+      return sb.toString();
+    }
+    return fs.pathSuffix;
+  }
+
+  private String humanReadableByteCount(long bytes) {
+    int unit = 1024;
+    if (bytes < unit) return bytes + " B";
+    int exp = (int) (Math.log(bytes) / Math.log(unit));
+    String pre = "KMGTPE".charAt(exp - 1) + "";
+    return String.format("%.1f %sB", bytes / Math.pow(unit, exp), pre);
+  }
+
+  public String listFile(String filePath) {
+    try {
+      String str = cmd.runCommand(cmd.getFileStatus, filePath, null);
+      SingleFileStatus sfs = gson.fromJson(str, SingleFileStatus.class);
+      if (sfs != null) {
+        return ListOne(filePath, sfs.FileStatus);
+      }
+    } catch (Exception e) {
+      logger.error("listFile: " + filePath, e);
+    }
+    return "No such File or directory";
+  }
+
+  public String listAll(String path) {
+    String all = "";
+    if (exceptionOnConnect != null)
+      return "Error connecting to provided endpoint.";
+    try {
+      //see if directory.
+      if (isDirectory(path)) {
+        String sfs = listDir(path);
+        if (sfs != null) {
+          AllFileStatus allFiles = gson.fromJson(sfs, AllFileStatus.class);
+
+          if (allFiles != null &&
+                  allFiles.FileStatuses != null &&
+                  allFiles.FileStatuses.FileStatus != null)
+          {
+            for (OneFileStatus fs : allFiles.FileStatuses.FileStatus)
+              all = all + ListOne(path, fs) + '\n';
+          }
+        }
+        return all;
+      } else {
+        return listFile(path);
+      }
+    } catch (Exception e) {
+      logger.error("listall: listDir " + path, e);
+      throw new InterpreterException("Could not find file or directory:\t" + path);
+    }
+  }
+
+  public boolean isDirectory(String path) {
+    boolean ret = false;
+    if (exceptionOnConnect != null)
+      return ret;
+    try {
+      String str = cmd.runCommand(cmd.getFileStatus, path, null);
+      SingleFileStatus sfs = gson.fromJson(str, SingleFileStatus.class);
+      if (sfs != null)
+        return sfs.FileStatus.type.equals("DIRECTORY");
+    } catch (Exception e) {
+      logger.error("IsDirectory: " + path, e);
+      return false;
+    }
+    return ret;
+  }
+
+
+  @Override
+  public List<String> completion(String buf, int cursor) {
+    logger.info("Completion request at position\t" + cursor + " in string " + buf);
+    final List<String> suggestions = new ArrayList<>();
+    if (StringUtils.isEmpty(buf)) {
+      suggestions.add("ls");
+      suggestions.add("cd");
+      suggestions.add("pwd");
+      return suggestions;
+    }
+
+    //part of a command == no spaces
+    if (buf.split(" ").length == 1){
+      if ("cd".contains(buf)) suggestions.add("cd");
+      if ("ls".contains(buf)) suggestions.add("ls");
+      if ("pwd".contains(buf)) suggestions.add("pwd");
+
+      return suggestions;
+    }
+
+
+    // last word will contain the path we're working with.
+    String lastToken = buf.substring(buf.lastIndexOf(" ") + 1);
+    if (lastToken.startsWith("-")) { //flag not path
+      return null;
+    }
+
+    String localPath = ""; //all things before the last '/'
+    String unfinished = lastToken; //unfished filenames or directories
+    if (lastToken.contains("/")) {
+      localPath = lastToken.substring(0, lastToken.lastIndexOf('/') + 1);
+      unfinished = lastToken.substring(lastToken.lastIndexOf('/') + 1);
+    }
+    String globalPath = getNewPath(localPath); //adjust for cwd
+
+    if (isDirectory(globalPath)){
+      try {
+        String fileStatusString = listDir(globalPath);
+        if (fileStatusString != null) {
+          AllFileStatus allFiles = gson.fromJson(fileStatusString, AllFileStatus.class);
+
+          if (allFiles != null &&
+                  allFiles.FileStatuses != null &&
+                  allFiles.FileStatuses.FileStatus != null)
+          {
+            for (OneFileStatus fs : allFiles.FileStatuses.FileStatus) {
+              if (fs.pathSuffix.contains(unfinished)) {
+
+                //only suggest the text after the last .
+                String beforeLastPeriod = unfinished.substring(0, unfinished.lastIndexOf('.') + 1);
+                //beforeLastPeriod should be the start of fs.pathSuffix, so take the end of it.
+                String suggestedFinish = fs.pathSuffix.substring(beforeLastPeriod.length());
+                suggestions.add(suggestedFinish);
+              }
+            }
+            return suggestions;
+          }
+        }
+      } catch (Exception e) {
+        logger.error("listall: listDir " + globalPath, e);
+        return null;
+      }
+    } else {
+      logger.info("path is not a directory.  No values suggested.");
+    }
+
+    //Error in string.
+    return null;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-zeppelin/blob/b45663d2/file/src/test/java/org/apache/zeppelin/file/HDFSFileInterpreterTest.java
----------------------------------------------------------------------
diff --git a/file/src/test/java/org/apache/zeppelin/file/HDFSFileInterpreterTest.java b/file/src/test/java/org/apache/zeppelin/file/HDFSFileInterpreterTest.java
new file mode 100644
index 0000000..3c87fa6
--- /dev/null
+++ b/file/src/test/java/org/apache/zeppelin/file/HDFSFileInterpreterTest.java
@@ -0,0 +1,209 @@
+/**
+ * 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.file;
+
+import com.google.gson.Gson;
+import junit.framework.TestCase;
+import static org.junit.Assert.*;
+import org.apache.zeppelin.interpreter.InterpreterResult;
+import org.junit.Test;
+import org.slf4j.Logger;
+import java.util.HashMap;
+import java.util.Properties;
+import java.lang.Override;
+import java.lang.String;
+
+
+/**
+ * Tests Interpreter by running pre-determined commands against mock file system
+ *
+ */
+public class HDFSFileInterpreterTest extends TestCase {
+
+    @Test
+    public void test() {
+      HDFSFileInterpreter t = new MockHDFSFileInterpreter(new Properties());
+      t.open();
+
+      // We have info for /, /user, /tmp, /mr-history/done
+
+      // Ensure
+      // 1. ls -l works
+      // 2. paths (. and ..) are correctly handled
+      // 3. flags and arguments to commands are correctly handled
+
+      InterpreterResult result1 = t.interpret("ls -l /", null);
+      assertEquals(result1.type(), InterpreterResult.Type.TEXT);
+
+      InterpreterResult result2 = t.interpret("ls -l /./user/..", null);
+      assertEquals(result2.type(), InterpreterResult.Type.TEXT);
+
+      assertEquals(result1.message(), result2.message());
+
+      // Ensure you can do cd and after that the ls uses current directory correctly
+
+      InterpreterResult result3 = t.interpret("cd user", null);
+      assertEquals(result3.type(), InterpreterResult.Type.TEXT);
+      assertEquals(result3.message(), "OK");
+
+      InterpreterResult result4 = t.interpret("ls", null);
+      assertEquals(result4.type(), InterpreterResult.Type.TEXT);
+
+      InterpreterResult result5 = t.interpret("ls /user", null);
+      assertEquals(result5.type(), InterpreterResult.Type.TEXT);
+
+      assertEquals(result4.message(), result5.message());
+
+      // Ensure pwd works correctly
+
+      InterpreterResult result6 = t.interpret("pwd", null);
+      assertEquals(result6.type(), InterpreterResult.Type.TEXT);
+      assertEquals(result6.message(), "/user");
+
+      // Move a couple of levels and check we're in the right place
+
+      InterpreterResult result7 = t.interpret("cd ../mr-history/done", null);
+      assertEquals(result7.type(), InterpreterResult.Type.TEXT);
+      assertEquals(result7.message(), "OK");
+
+      InterpreterResult result8 = t.interpret("ls -l ", null);
+      assertEquals(result8.type(), InterpreterResult.Type.TEXT);
+
+      InterpreterResult result9 = t.interpret("ls -l /mr-history/done", null);
+      assertEquals(result9.type(), InterpreterResult.Type.TEXT);
+
+      assertEquals(result8.message(), result9.message());
+
+      InterpreterResult result10 = t.interpret("cd ../..", null);
+      assertEquals(result10.type(), InterpreterResult.Type.TEXT);
+      assertEquals(result7.message(), "OK");
+
+      InterpreterResult result11 = t.interpret("ls -l ", null);
+      assertEquals(result11.type(), InterpreterResult.Type.TEXT);
+
+      // we should be back to first result after all this navigation
+      assertEquals(result1.message(), result11.message());
+
+      t.close();
+    }
+  }
+
+  /**
+   * Store command results from curl against a real file system
+   */
+  class MockFileSystem {
+    HashMap<String, String> mfs = new HashMap<String, String>();
+    void addListStatusData() {
+      mfs.put("/?op=LISTSTATUS",
+          "{\"FileStatuses\":{\"FileStatus\":[\n" +
+              "{\"accessTime\":0,\"blockSize\":0,\"childrenNum\":1,\"fileId\":16389,\"group\":\"hadoop\",\"length\":0,\"modificationTime\":1438548219672,\"owner\":\"yarn\",\"pathSuffix\":\"app-logs\",\"permission\":\"777\",\"replication\":0,\"storagePolicy\":0,\"type\":\"DIRECTORY\"},\n" +
+              "{\"accessTime\":0,\"blockSize\":0,\"childrenNum\":1,\"fileId\":16395,\"group\":\"hdfs\",\"length\":0,\"modificationTime\":1438548030045,\"owner\":\"hdfs\",\"pathSuffix\":\"hdp\",\"permission\":\"755\",\"replication\":0,\"storagePolicy\":0,\"type\":\"DIRECTORY\"},\n" +
+              "{\"accessTime\":0,\"blockSize\":0,\"childrenNum\":1,\"fileId\":16390,\"group\":\"hdfs\",\"length\":0,\"modificationTime\":1438547985336,\"owner\":\"mapred\",\"pathSuffix\":\"mapred\",\"permission\":\"755\",\"replication\":0,\"storagePolicy\":0,\"type\":\"DIRECTORY\"},\n" +
+              "{\"accessTime\":0,\"blockSize\":0,\"childrenNum\":2,\"fileId\":16392,\"group\":\"hdfs\",\"length\":0,\"modificationTime\":1438547985346,\"owner\":\"hdfs\",\"pathSuffix\":\"mr-history\",\"permission\":\"755\",\"replication\":0,\"storagePolicy\":0,\"type\":\"DIRECTORY\"},\n" +
+              "{\"accessTime\":0,\"blockSize\":0,\"childrenNum\":1,\"fileId\":16400,\"group\":\"hdfs\",\"length\":0,\"modificationTime\":1438548089725,\"owner\":\"hdfs\",\"pathSuffix\":\"system\",\"permission\":\"755\",\"replication\":0,\"storagePolicy\":0,\"type\":\"DIRECTORY\"},\n" +
+              "{\"accessTime\":0,\"blockSize\":0,\"childrenNum\":1,\"fileId\":16386,\"group\":\"hdfs\",\"length\":0,\"modificationTime\":1438548150089,\"owner\":\"hdfs\",\"pathSuffix\":\"tmp\",\"permission\":\"777\",\"replication\":0,\"storagePolicy\":0,\"type\":\"DIRECTORY\"},\n" +
+              "{\"accessTime\":0,\"blockSize\":0,\"childrenNum\":1,\"fileId\":16387,\"group\":\"hdfs\",\"length\":0,\"modificationTime\":1438547921792,\"owner\":\"hdfs\",\"pathSuffix\":\"user\",\"permission\":\"755\",\"replication\":0,\"storagePolicy\":0,\"type\":\"DIRECTORY\"}\n" +
+              "]}}"
+      );
+      mfs.put("/user?op=LISTSTATUS",
+         "{\"FileStatuses\":{\"FileStatus\":[\n" +
+             "        {\"accessTime\":0,\"blockSize\":0,\"childrenNum\":4,\"fileId\":16388,\"group\":\"hdfs\",\"length\":0,\"modificationTime\":1441253161263,\"owner\":\"ambari-qa\",\"pathSuffix\":\"ambari-qa\",\"permission\":\"770\",\"replication\":0,\"storagePolicy\":0,\"type\":\"DIRECTORY\"}\n" +
+             "        ]}}"
+      );
+      mfs.put("/tmp?op=LISTSTATUS",
+          "{\"FileStatuses\":{\"FileStatus\":[\n" +
+              "        {\"accessTime\":1441253097489,\"blockSize\":134217728,\"childrenNum\":0,\"fileId\":16400,\"group\":\"hdfs\",\"length\":1645,\"modificationTime\":1441253097517,\"owner\":\"hdfs\",\"pathSuffix\":\"ida8c06540_date040315\",\"permission\":\"755\",\"replication\":3,\"storagePolicy\":0,\"type\":\"FILE\"}\n" +
+              "        ]}}"
+      );
+      mfs.put("/mr-history/done?op=LISTSTATUS",
+          "{\"FileStatuses\":{\"FileStatus\":[\n" +
+          "{\"accessTime\":0,\"blockSize\":0,\"childrenNum\":1,\"fileId\":16433,\"group\":\"hadoop\",\"length\":0,\"modificationTime\":1441253197481,\"owner\":\"mapred\",\"pathSuffix\":\"2015\",\"permission\":\"770\",\"replication\":0,\"storagePolicy\":0,\"type\":\"DIRECTORY\"}\n" +
+          "]}}"
+      );
+    }
+    void addGetFileStatusData() {
+      mfs.put("/?op=GETFILESTATUS",
+          "{\"FileStatus\":{\"accessTime\":0,\"blockSize\":0,\"childrenNum\":7,\"fileId\":16385,\"group\":\"hdfs\",\"length\":0,\"modificationTime\":1438548089725,\"owner\":\"hdfs\",\"pathSuffix\":\"\",\"permission\":\"755\",\"replication\":0,\"storagePolicy\":0,\"type\":\"DIRECTORY\"}}");
+      mfs.put("/user?op=GETFILESTATUS",
+          "{\"FileStatus\":{\"accessTime\":0,\"blockSize\":0,\"childrenNum\":1,\"fileId\":16387,\"group\":\"hdfs\",\"length\":0,\"modificationTime\":1441253043188,\"owner\":\"hdfs\",\"pathSuffix\":\"\",\"permission\":\"755\",\"replication\":0,\"storagePolicy\":0,\"type\":\"DIRECTORY\"}}");
+      mfs.put("/tmp?op=GETFILESTATUS",
+          "{\"FileStatus\":{\"accessTime\":0,\"blockSize\":0,\"childrenNum\":1,\"fileId\":16386,\"group\":\"hdfs\",\"length\":0,\"modificationTime\":1441253097489,\"owner\":\"hdfs\",\"pathSuffix\":\"\",\"permission\":\"777\",\"replication\":0,\"storagePolicy\":0,\"type\":\"DIRECTORY\"}}");
+      mfs.put("/mr-history/done?op=GETFILESTATUS",
+          "{\"FileStatus\":{\"accessTime\":0,\"blockSize\":0,\"childrenNum\":1,\"fileId\":16393,\"group\":\"hadoop\",\"length\":0,\"modificationTime\":1441253197480,\"owner\":\"mapred\",\"pathSuffix\":\"\",\"permission\":\"777\",\"replication\":0,\"storagePolicy\":0,\"type\":\"DIRECTORY\"}}");
+    }
+    public void addMockData(HDFSCommand.Op op) {
+      if (op.op.equals("LISTSTATUS")) {
+        addListStatusData();
+      } else if (op.op.equals("GETFILESTATUS")) {
+        addGetFileStatusData();
+      }
+      // do nothing
+    }
+    public String get(String key) {
+      return mfs.get(key);
+    }
+  }
+
+  /**
+   * Run commands against mock file system that simulates webhdfs responses
+   */
+  class MockHDFSCommand extends HDFSCommand {
+    MockFileSystem fs = null;
+
+    public MockHDFSCommand(String url, String user, Logger logger) {
+      super(url, user, logger, 1000);
+      fs = new MockFileSystem();
+      fs.addMockData(getFileStatus);
+      fs.addMockData(listStatus);
+    }
+
+    @Override
+    public String runCommand(Op op, String path, Arg[] args) throws Exception {
+
+      String error = checkArgs(op, path, args);
+      assertNull(error);
+
+      String c = path + "?op=" + op.op;
+
+      if (args != null) {
+        for (Arg a : args) {
+          c += "&" + a.key + "=" + a.value;
+        }
+      }
+      return fs.get(c);
+    }
+  }
+
+  /**
+   * Mock Interpreter - uses Mock HDFS command
+   */
+  class MockHDFSFileInterpreter extends HDFSFileInterpreter {
+
+    @Override
+    public void prepare() {
+      // Run commands against mock File System instead of WebHDFS
+      cmd = new MockHDFSCommand("", "", logger);
+      gson = new Gson();
+    }
+
+    public MockHDFSFileInterpreter(Properties property) {
+      super(property);
+    }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-zeppelin/blob/b45663d2/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index d173e89..f566cca 100755
--- a/pom.xml
+++ b/pom.xml
@@ -98,6 +98,7 @@
     <module>postgresql</module>
     <module>jdbc</module>
     <module>tajo</module>
+    <module>file</module>
     <module>flink</module>
     <module>ignite</module>
     <module>kylin</module>

http://git-wip-us.apache.org/repos/asf/incubator-zeppelin/blob/b45663d2/zeppelin-distribution/src/bin_license/LICENSE
----------------------------------------------------------------------
diff --git a/zeppelin-distribution/src/bin_license/LICENSE b/zeppelin-distribution/src/bin_license/LICENSE
index a8dc0ed..84289c0 100644
--- a/zeppelin-distribution/src/bin_license/LICENSE
+++ b/zeppelin-distribution/src/bin_license/LICENSE
@@ -57,6 +57,7 @@ The following components are provided under Apache License.
     (Apache 2.0) javax.servlet (org.eclipse.jetty.orbit:javax.servlet:jar:3.0.0.v201112011016 - http://www.eclipse.org/jetty)
     (Apache 2.0) Joda-Time (joda-time:joda-time:2.8.1 - http://www.joda.org/joda-time/)
     (Apache 2.0) Jackson (org.codehaus.jackson:jackson-core-asl:1.9.13 - http://jackson.codehaus.org/)
+    (Apache 2.0) Javassist (org.javassist:javassist:jar:3.18.1-GA:compile - http://jboss-javassist.github.io/javassist/)
     (Apache 2.0) JetS3t (net.java.dev.jets3t:jets3t:jar:0.9.3) - http://www.jets3t.org/
     (Apache 2.0) Jetty (org.eclipse.jetty:jetty - http://www.eclipse.org/jetty)
     (Apache 2.0) mx4j (mx4j:mx4j:jar:3.0.2) - http://mx4j.sourceforge.net/
@@ -187,7 +188,10 @@ CDDL license
 The following components are provided under the CDDL License.
 
     (CDDL 1.0) javax.activation (javax.activation:activation:jar:1.1.1 - http://java.sun.com/javase/technologies/desktop/javabeans/jaf/index.jsp)
+    (CDDL 1.0) java.annotation (javax.annotation:javax.annotation-api:jar:1.2:compile - http://jcp.org/en/jsr/detail?id=250)
     (CDDL 1.1) Jersey (com.sun.jersey:jersey:jar:1.9 - https://jersey.java.net/)
+    (CDDL 1.1) jersey-core (org.glassfish.jersey.core:jersey-core:2.22.2 - https://jersey.java.net/)
+    (CDDL 1.1) hk2 (org.glassfish.hk2 - https://hk2.java.net/2.5.0-b03/)
 
 
 

http://git-wip-us.apache.org/repos/asf/incubator-zeppelin/blob/b45663d2/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 9c365b2..25a9b12 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
@@ -450,6 +450,7 @@ public class ZeppelinConfiguration extends XMLConfiguration {
         + "org.apache.zeppelin.shell.ShellInterpreter,"
         + "org.apache.zeppelin.hive.HiveInterpreter,"
         + "org.apache.zeppelin.alluxio.AlluxioInterpreter,"
+        + "org.apache.zeppelin.file.HDFSFileInterpreter,"
         + "org.apache.zeppelin.phoenix.PhoenixInterpreter,"
         + "org.apache.zeppelin.postgresql.PostgreSqlInterpreter,"
         + "org.apache.zeppelin.tajo.TajoInterpreter,"