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 2017/02/10 01:18:28 UTC

zeppelin git commit: [ZEPPELIN-753] TableData abstraction

Repository: zeppelin
Updated Branches:
  refs/heads/master db691db99 -> 9413323ed


[ZEPPELIN-753] TableData abstraction

### What is this PR for?
This PR implements TableData abstraction that every interpreter can use to share data through resource pool.

TableData.java - Definition of TableData abstraction
TableDataProxy.java - Proxy object of TableData. This enables data access through ResourcePool
InterpreterResultTableData - Convert Table type interpret result into TableData. Each interpreter can implement it's own TableData abstraction. For example JDBCTableData, DataFrameTableData, and so on.

### What type of PR is it?
Feature

### Todos
* [x] - TableData
* [x] - TableDataProxy
* [x] - Unittest

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

### How should this be tested?
Unittest included

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

Author: Lee moon soo <mo...@apache.org>

Closes #1952 from Leemoonsoo/ZEPPELIN-753 and squashes the following commits:

d85964b [Lee moon soo] Proxy for tabledata
2211d30 [Lee moon soo] TableData abstraction
3b49105 [Lee moon soo] make RemoteResource.invokeMethod returns Resource when return resource name is specified
60a1dcb [Lee moon soo] Support method invocation on the resource in local,remote resource pool


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

Branch: refs/heads/master
Commit: 9413323ed13909810d55851eda19d599e731100a
Parents: db691db
Author: Lee moon soo <mo...@apache.org>
Authored: Wed Jan 25 18:38:45 2017 -0800
Committer: Lee moon soo <mo...@apache.org>
Committed: Fri Feb 10 10:18:17 2017 +0900

----------------------------------------------------------------------
 .../interpreter/InterpreterResultMessage.java   |  4 +-
 .../remote/RemoteInterpreterEventClient.java    |  1 -
 .../org/apache/zeppelin/resource/Resource.java  |  1 +
 .../apache/zeppelin/tabledata/ColumnDef.java    | 49 +++++++++++++++
 .../tabledata/InterpreterResultTableData.java   | 64 +++++++++++++++++++
 .../zeppelin/tabledata/ProxyRowIterator.java    | 49 +++++++++++++++
 .../java/org/apache/zeppelin/tabledata/Row.java | 34 ++++++++++
 .../apache/zeppelin/tabledata/TableData.java    | 37 +++++++++++
 .../zeppelin/tabledata/TableDataException.java  | 28 +++++++++
 .../zeppelin/tabledata/TableDataProxy.java      | 48 ++++++++++++++
 .../mock/MockInterpreterResourcePool.java       |  2 +-
 .../InterpreterResultTableDataTest.java         | 54 ++++++++++++++++
 .../zeppelin/tabledata/TableDataProxyTest.java  | 66 ++++++++++++++++++++
 13 files changed, 434 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/zeppelin/blob/9413323e/zeppelin-interpreter/src/main/java/org/apache/zeppelin/interpreter/InterpreterResultMessage.java
----------------------------------------------------------------------
diff --git a/zeppelin-interpreter/src/main/java/org/apache/zeppelin/interpreter/InterpreterResultMessage.java b/zeppelin-interpreter/src/main/java/org/apache/zeppelin/interpreter/InterpreterResultMessage.java
index 20d9951..f137ca5 100644
--- a/zeppelin-interpreter/src/main/java/org/apache/zeppelin/interpreter/InterpreterResultMessage.java
+++ b/zeppelin-interpreter/src/main/java/org/apache/zeppelin/interpreter/InterpreterResultMessage.java
@@ -16,10 +16,12 @@
  */
 package org.apache.zeppelin.interpreter;
 
+import java.io.Serializable;
+
 /**
  * Interpreter result message
  */
-public class InterpreterResultMessage {
+public class InterpreterResultMessage implements Serializable {
   InterpreterResult.Type type;
   String data;
 

http://git-wip-us.apache.org/repos/asf/zeppelin/blob/9413323e/zeppelin-interpreter/src/main/java/org/apache/zeppelin/interpreter/remote/RemoteInterpreterEventClient.java
----------------------------------------------------------------------
diff --git a/zeppelin-interpreter/src/main/java/org/apache/zeppelin/interpreter/remote/RemoteInterpreterEventClient.java b/zeppelin-interpreter/src/main/java/org/apache/zeppelin/interpreter/remote/RemoteInterpreterEventClient.java
index 4b721f5..73c14f1 100644
--- a/zeppelin-interpreter/src/main/java/org/apache/zeppelin/interpreter/remote/RemoteInterpreterEventClient.java
+++ b/zeppelin-interpreter/src/main/java/org/apache/zeppelin/interpreter/remote/RemoteInterpreterEventClient.java
@@ -351,7 +351,6 @@ public class RemoteInterpreterEventClient implements ResourcePoolConnector {
     }
   }
 
-
   /**
    * Supposed to call from RemoteInterpreterEventPoller
    * @return next available event

http://git-wip-us.apache.org/repos/asf/zeppelin/blob/9413323e/zeppelin-interpreter/src/main/java/org/apache/zeppelin/resource/Resource.java
----------------------------------------------------------------------
diff --git a/zeppelin-interpreter/src/main/java/org/apache/zeppelin/resource/Resource.java b/zeppelin-interpreter/src/main/java/org/apache/zeppelin/resource/Resource.java
index a478c42..098e7ba 100644
--- a/zeppelin-interpreter/src/main/java/org/apache/zeppelin/resource/Resource.java
+++ b/zeppelin-interpreter/src/main/java/org/apache/zeppelin/resource/Resource.java
@@ -117,6 +117,7 @@ public class Resource {
         Method method = r.getClass().getMethod(
             methodName,
             paramTypes);
+        method.setAccessible(true);
         Object ret = method.invoke(r, params);
         return ret;
       }  catch (Exception e) {

http://git-wip-us.apache.org/repos/asf/zeppelin/blob/9413323e/zeppelin-interpreter/src/main/java/org/apache/zeppelin/tabledata/ColumnDef.java
----------------------------------------------------------------------
diff --git a/zeppelin-interpreter/src/main/java/org/apache/zeppelin/tabledata/ColumnDef.java b/zeppelin-interpreter/src/main/java/org/apache/zeppelin/tabledata/ColumnDef.java
new file mode 100644
index 0000000..75afc18
--- /dev/null
+++ b/zeppelin-interpreter/src/main/java/org/apache/zeppelin/tabledata/ColumnDef.java
@@ -0,0 +1,49 @@
+/*
+ * 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.tabledata;
+
+import java.io.Serializable;
+
+/**
+ * Column definition
+ */
+public class ColumnDef implements Serializable {
+  /**
+   * Type
+   */
+  public static enum TYPE {
+    STRING,
+    LONG,
+    INT
+  }
+
+  private String name;
+  private TYPE type;
+
+  public ColumnDef(String name, TYPE type) {
+    this.name = name;
+    this.type = type;
+  }
+
+  public String name() {
+    return name;
+  }
+
+  public TYPE type() {
+    return type;
+  }
+}

http://git-wip-us.apache.org/repos/asf/zeppelin/blob/9413323e/zeppelin-interpreter/src/main/java/org/apache/zeppelin/tabledata/InterpreterResultTableData.java
----------------------------------------------------------------------
diff --git a/zeppelin-interpreter/src/main/java/org/apache/zeppelin/tabledata/InterpreterResultTableData.java b/zeppelin-interpreter/src/main/java/org/apache/zeppelin/tabledata/InterpreterResultTableData.java
new file mode 100644
index 0000000..e11ad45
--- /dev/null
+++ b/zeppelin-interpreter/src/main/java/org/apache/zeppelin/tabledata/InterpreterResultTableData.java
@@ -0,0 +1,64 @@
+/*
+ * 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.tabledata;
+
+import org.apache.zeppelin.interpreter.InterpreterResultMessage;
+
+import java.io.Serializable;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+
+/**
+ * Table data with interpreter result type 'TABLE'
+ */
+public class InterpreterResultTableData implements TableData, Serializable {
+  private final InterpreterResultMessage msg;
+  ColumnDef [] columnDef;
+  List<Row> rows = new LinkedList<>();
+
+  public InterpreterResultTableData(InterpreterResultMessage msg) {
+    this.msg = msg;
+
+    String[] lines = msg.getData().split("\n");
+    if (lines == null || lines.length == 0) {
+      columnDef = null;
+    } else {
+      String[] headerRow = lines[0].split("\t");
+      columnDef = new ColumnDef[headerRow.length];
+      for (int i = 0; i < headerRow.length; i++) {
+        columnDef[i] = new ColumnDef(headerRow[i], ColumnDef.TYPE.STRING);
+      }
+
+      for (int r = 1; r < lines.length; r++) {
+        Object [] row = lines[r].split("\t");
+        rows.add(new Row(row));
+      }
+    }
+  }
+
+
+  @Override
+  public ColumnDef[] columns() {
+    return columnDef;
+  }
+
+  @Override
+  public Iterator<Row> rows() {
+    return rows.iterator();
+  }
+}

http://git-wip-us.apache.org/repos/asf/zeppelin/blob/9413323e/zeppelin-interpreter/src/main/java/org/apache/zeppelin/tabledata/ProxyRowIterator.java
----------------------------------------------------------------------
diff --git a/zeppelin-interpreter/src/main/java/org/apache/zeppelin/tabledata/ProxyRowIterator.java b/zeppelin-interpreter/src/main/java/org/apache/zeppelin/tabledata/ProxyRowIterator.java
new file mode 100644
index 0000000..8a59098
--- /dev/null
+++ b/zeppelin-interpreter/src/main/java/org/apache/zeppelin/tabledata/ProxyRowIterator.java
@@ -0,0 +1,49 @@
+/*
+ * 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.tabledata;
+
+import org.apache.zeppelin.resource.Resource;
+
+import java.util.Iterator;
+
+/**
+ * Proxy row iterator
+ */
+public class ProxyRowIterator implements Iterator<Row> {
+
+  private final Resource rows;
+
+  public ProxyRowIterator(Resource rows) {
+    this.rows = rows;
+  }
+
+  @Override
+  public boolean hasNext() {
+    rows.invokeMethod("hasNext", null, null);
+    return false;
+  }
+
+  @Override
+  public Row next() {
+    return (Row) rows.invokeMethod("next", null, null);
+  }
+
+  @Override
+  public void remove() {
+    // operation not supported
+  }
+}

http://git-wip-us.apache.org/repos/asf/zeppelin/blob/9413323e/zeppelin-interpreter/src/main/java/org/apache/zeppelin/tabledata/Row.java
----------------------------------------------------------------------
diff --git a/zeppelin-interpreter/src/main/java/org/apache/zeppelin/tabledata/Row.java b/zeppelin-interpreter/src/main/java/org/apache/zeppelin/tabledata/Row.java
new file mode 100644
index 0000000..9dcf2db
--- /dev/null
+++ b/zeppelin-interpreter/src/main/java/org/apache/zeppelin/tabledata/Row.java
@@ -0,0 +1,34 @@
+/*
+ * 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.tabledata;
+
+import java.io.Serializable;
+
+/**
+ * Row representation of table data
+ */
+public class Row implements Serializable {
+  private final Object[] data;
+
+  public Row(Object [] data) {
+    this.data = data;
+  }
+
+  public Object [] get() {
+    return data;
+  }
+}

http://git-wip-us.apache.org/repos/asf/zeppelin/blob/9413323e/zeppelin-interpreter/src/main/java/org/apache/zeppelin/tabledata/TableData.java
----------------------------------------------------------------------
diff --git a/zeppelin-interpreter/src/main/java/org/apache/zeppelin/tabledata/TableData.java b/zeppelin-interpreter/src/main/java/org/apache/zeppelin/tabledata/TableData.java
new file mode 100644
index 0000000..6e34f47
--- /dev/null
+++ b/zeppelin-interpreter/src/main/java/org/apache/zeppelin/tabledata/TableData.java
@@ -0,0 +1,37 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.zeppelin.tabledata;
+
+import java.util.Iterator;
+
+/**
+ * Abstract representation of table data
+ */
+public interface TableData {
+  /**
+   * Get column definitions
+   * @return
+   */
+  public ColumnDef [] columns();
+
+  /**
+   * Get row iterator
+   * @param
+   * @return
+   */
+  public Iterator<Row> rows();
+}

http://git-wip-us.apache.org/repos/asf/zeppelin/blob/9413323e/zeppelin-interpreter/src/main/java/org/apache/zeppelin/tabledata/TableDataException.java
----------------------------------------------------------------------
diff --git a/zeppelin-interpreter/src/main/java/org/apache/zeppelin/tabledata/TableDataException.java b/zeppelin-interpreter/src/main/java/org/apache/zeppelin/tabledata/TableDataException.java
new file mode 100644
index 0000000..d465f1a
--- /dev/null
+++ b/zeppelin-interpreter/src/main/java/org/apache/zeppelin/tabledata/TableDataException.java
@@ -0,0 +1,28 @@
+/*
+ * 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.tabledata;
+
+import java.io.IOException;
+
+/**
+ * TableDataException
+ */
+public class TableDataException extends IOException {
+  public TableDataException(String s) {
+    super(s);
+  }
+}

http://git-wip-us.apache.org/repos/asf/zeppelin/blob/9413323e/zeppelin-interpreter/src/main/java/org/apache/zeppelin/tabledata/TableDataProxy.java
----------------------------------------------------------------------
diff --git a/zeppelin-interpreter/src/main/java/org/apache/zeppelin/tabledata/TableDataProxy.java b/zeppelin-interpreter/src/main/java/org/apache/zeppelin/tabledata/TableDataProxy.java
new file mode 100644
index 0000000..8673476
--- /dev/null
+++ b/zeppelin-interpreter/src/main/java/org/apache/zeppelin/tabledata/TableDataProxy.java
@@ -0,0 +1,48 @@
+/*
+ * 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.tabledata;
+
+import org.apache.zeppelin.resource.Resource;
+import org.apache.zeppelin.resource.ResourcePoolUtils;
+
+import java.util.Iterator;
+
+/**
+ * Proxy TableData for ResourcePool
+ */
+public class TableDataProxy implements TableData {
+  private final Resource resource;
+
+  public TableDataProxy(Resource tableDataRemoteResource) {
+    this.resource = tableDataRemoteResource;
+  }
+
+  @Override
+  public ColumnDef[] columns() {
+    return (ColumnDef[]) resource.invokeMethod(
+        "columns", null, null);
+  }
+
+  @Override
+  public Iterator<Row> rows() {
+    String resourceName = resource.getResourceId().getName() + ".rows";
+    Resource rows = resource.invokeMethod("rows", null, null, resourceName);
+
+    ProxyRowIterator it = new ProxyRowIterator(rows);
+    return it;
+  }
+}

http://git-wip-us.apache.org/repos/asf/zeppelin/blob/9413323e/zeppelin-interpreter/src/test/java/org/apache/zeppelin/interpreter/remote/mock/MockInterpreterResourcePool.java
----------------------------------------------------------------------
diff --git a/zeppelin-interpreter/src/test/java/org/apache/zeppelin/interpreter/remote/mock/MockInterpreterResourcePool.java b/zeppelin-interpreter/src/test/java/org/apache/zeppelin/interpreter/remote/mock/MockInterpreterResourcePool.java
index 80ac555..be746bd 100644
--- a/zeppelin-interpreter/src/test/java/org/apache/zeppelin/interpreter/remote/mock/MockInterpreterResourcePool.java
+++ b/zeppelin-interpreter/src/test/java/org/apache/zeppelin/interpreter/remote/mock/MockInterpreterResourcePool.java
@@ -133,4 +133,4 @@ public class MockInterpreterResourcePool extends Interpreter {
   public List<InterpreterCompletion> completion(String buf, int cursor) {
     return null;
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/zeppelin/blob/9413323e/zeppelin-interpreter/src/test/java/org/apache/zeppelin/tabledata/InterpreterResultTableDataTest.java
----------------------------------------------------------------------
diff --git a/zeppelin-interpreter/src/test/java/org/apache/zeppelin/tabledata/InterpreterResultTableDataTest.java b/zeppelin-interpreter/src/test/java/org/apache/zeppelin/tabledata/InterpreterResultTableDataTest.java
new file mode 100644
index 0000000..0618d3b
--- /dev/null
+++ b/zeppelin-interpreter/src/test/java/org/apache/zeppelin/tabledata/InterpreterResultTableDataTest.java
@@ -0,0 +1,54 @@
+/*
+ * 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.tabledata;
+
+import org.apache.zeppelin.interpreter.InterpreterResult;
+import org.apache.zeppelin.interpreter.InterpreterResultMessage;
+import org.junit.Test;
+
+import java.util.Iterator;
+
+import static junit.framework.TestCase.assertFalse;
+import static org.junit.Assert.assertEquals;
+
+public class InterpreterResultTableDataTest {
+  @Test
+  public void test() {
+    InterpreterResultMessage msg = new InterpreterResultMessage(
+        InterpreterResult.Type.TABLE,
+        "key\tvalue\nsun\t100\nmoon\t200\n");
+    InterpreterResultTableData table = new InterpreterResultTableData(msg);
+
+    ColumnDef[] cols = table.columns();
+    assertEquals(2, cols.length);
+
+    assertEquals("key", cols[0].name());
+    assertEquals("value", cols[1].name());
+
+    Iterator<Row> it = table.rows();
+    Row row = it.next();
+    assertEquals(2, row.get().length);
+    assertEquals("sun", row.get()[0]);
+    assertEquals("100", row.get()[1]);
+
+    row = it.next();
+    assertEquals("moon", row.get()[0]);
+    assertEquals("200", row.get()[1]);
+
+    assertFalse(it.hasNext());
+  }
+}

http://git-wip-us.apache.org/repos/asf/zeppelin/blob/9413323e/zeppelin-interpreter/src/test/java/org/apache/zeppelin/tabledata/TableDataProxyTest.java
----------------------------------------------------------------------
diff --git a/zeppelin-interpreter/src/test/java/org/apache/zeppelin/tabledata/TableDataProxyTest.java b/zeppelin-interpreter/src/test/java/org/apache/zeppelin/tabledata/TableDataProxyTest.java
new file mode 100644
index 0000000..a4a8ffe
--- /dev/null
+++ b/zeppelin-interpreter/src/test/java/org/apache/zeppelin/tabledata/TableDataProxyTest.java
@@ -0,0 +1,66 @@
+/*
+ * 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.tabledata;
+
+import org.apache.zeppelin.interpreter.InterpreterResult;
+import org.apache.zeppelin.interpreter.InterpreterResultMessage;
+import org.apache.zeppelin.resource.LocalResourcePool;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.util.Iterator;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+
+public class TableDataProxyTest {
+  private LocalResourcePool pool;
+
+  @Before
+  public void setUp() {
+    pool = new LocalResourcePool("p1");
+  }
+
+  @Test
+  public void testProxyTable() {
+    InterpreterResultMessage msg = new InterpreterResultMessage(
+        InterpreterResult.Type.TABLE,
+        "key\tvalue\nsun\t100\nmoon\t200\n");
+    InterpreterResultTableData table = new InterpreterResultTableData(msg);
+
+    pool.put("table", table);
+    TableDataProxy proxy = new TableDataProxy(pool.get("table"));
+
+    ColumnDef[] cols = proxy.columns();
+    assertEquals(2, cols.length);
+
+    assertEquals("key", cols[0].name());
+    assertEquals("value", cols[1].name());
+
+    Iterator<Row> it = proxy.rows();
+    Row row = it.next();
+    assertEquals(2, row.get().length);
+    assertEquals("sun", row.get()[0]);
+    assertEquals("100", row.get()[1]);
+
+    row = it.next();
+    assertEquals("moon", row.get()[0]);
+    assertEquals("200", row.get()[1]);
+
+    assertFalse(it.hasNext());
+  }
+}