You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@calcite.apache.org by jh...@apache.org on 2014/09/20 01:33:22 UTC

git commit: [OPTIQ-416] Execute logical RelNodes using an interpreter

Repository: incubator-optiq
Updated Branches:
  refs/heads/master 3b447fcc2 -> 5fba93bf6


[OPTIQ-416] Execute logical RelNodes using an interpreter


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

Branch: refs/heads/master
Commit: 5fba93bf6b0926418878a6f82b263378ab88dc4e
Parents: 3b447fc
Author: Julian Hyde <jh...@apache.org>
Authored: Thu Sep 18 01:23:06 2014 -0700
Committer: Julian Hyde <jh...@apache.org>
Committed: Fri Sep 19 15:46:51 2014 -0700

----------------------------------------------------------------------
 .../main/java/net/hydromatic/optiq/Schemas.java |  19 +-
 .../optiq/impl/ReflectiveFunctionBase.java      |   2 +-
 .../optiq/impl/interpreter/Context.java         |  27 ++
 .../optiq/impl/interpreter/FilterNode.java      |  49 ++++
 .../optiq/impl/interpreter/Interpreter.java     | 269 +++++++++++++++++++
 .../hydromatic/optiq/impl/interpreter/Node.java |  26 ++
 .../optiq/impl/interpreter/Nodes.java           |  57 ++++
 .../optiq/impl/interpreter/ProjectNode.java     |  58 ++++
 .../hydromatic/optiq/impl/interpreter/Row.java  |  55 ++++
 .../optiq/impl/interpreter/Scalar.java          |  26 ++
 .../optiq/impl/interpreter/ScanNode.java        | 116 ++++++++
 .../hydromatic/optiq/impl/interpreter/Sink.java |  30 +++
 .../optiq/impl/interpreter/SortNode.java        | 131 +++++++++
 .../optiq/impl/interpreter/Source.java          |  29 ++
 .../optiq/impl/interpreter/ValuesNode.java      |  51 ++++
 .../optiq/impl/interpreter/package-info.java    |  27 ++
 .../optiq/jdbc/JavaTypeFactoryImpl.java         |   2 +-
 .../hydromatic/optiq/test/InterpreterTest.java  | 136 ++++++++++
 .../net/hydromatic/optiq/test/OptiqSuite.java   |   1 +
 19 files changed, 1104 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-optiq/blob/5fba93bf/core/src/main/java/net/hydromatic/optiq/Schemas.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/net/hydromatic/optiq/Schemas.java b/core/src/main/java/net/hydromatic/optiq/Schemas.java
index 4d9a426..4c83bef 100644
--- a/core/src/main/java/net/hydromatic/optiq/Schemas.java
+++ b/core/src/main/java/net/hydromatic/optiq/Schemas.java
@@ -171,13 +171,22 @@ public final class Schemas {
   /** Returns a {@link Queryable}, given a fully-qualified table name. */
   public static <E> Queryable<E> queryable(DataContext root, Class<E> clazz,
       String... names) {
+    return queryable(root, clazz, Arrays.asList(names));
+  }
+
+  /** Returns a {@link Queryable}, given a fully-qualified table name as an
+   * iterable. */
+  public static <E> Queryable<E> queryable(DataContext root, Class<E> clazz,
+      Iterable<? extends String> names) {
     SchemaPlus schema = root.getRootSchema();
-    for (int i = 0; i < names.length - 1; i++) {
-      String name = names[i];
-      schema = schema.getSubSchema(name);
+    for (Iterator<? extends String> iterator = names.iterator();;) {
+      String name = iterator.next();
+      if (iterator.hasNext()) {
+        schema = schema.getSubSchema(name);
+      } else {
+        return queryable(root, schema, clazz, name);
+      }
     }
-    final String tableName = names[names.length - 1];
-    return queryable(root, schema, clazz, tableName);
   }
 
   /** Returns a {@link Queryable}, given a schema and table name. */

http://git-wip-us.apache.org/repos/asf/incubator-optiq/blob/5fba93bf/core/src/main/java/net/hydromatic/optiq/impl/ReflectiveFunctionBase.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/net/hydromatic/optiq/impl/ReflectiveFunctionBase.java b/core/src/main/java/net/hydromatic/optiq/impl/ReflectiveFunctionBase.java
index 140866d..12e8f90 100644
--- a/core/src/main/java/net/hydromatic/optiq/impl/ReflectiveFunctionBase.java
+++ b/core/src/main/java/net/hydromatic/optiq/impl/ReflectiveFunctionBase.java
@@ -98,7 +98,7 @@ public abstract class ReflectiveFunctionBase implements Function {
   static boolean classHasPublicZeroArgsConstructor(Class<?> clazz) {
     for (Constructor<?> constructor : clazz.getConstructors()) {
       if (constructor.getParameterTypes().length == 0
-          && (constructor.getModifiers() & Modifier.PUBLIC) != 0) {
+          && Modifier.isPublic(constructor.getModifiers())) {
         return true;
       }
     }

http://git-wip-us.apache.org/repos/asf/incubator-optiq/blob/5fba93bf/core/src/main/java/net/hydromatic/optiq/impl/interpreter/Context.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/net/hydromatic/optiq/impl/interpreter/Context.java b/core/src/main/java/net/hydromatic/optiq/impl/interpreter/Context.java
new file mode 100644
index 0000000..3d2b2f1
--- /dev/null
+++ b/core/src/main/java/net/hydromatic/optiq/impl/interpreter/Context.java
@@ -0,0 +1,27 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package net.hydromatic.optiq.impl.interpreter;
+
+/**
+ * Context for executing a scalar expression in an interpreter.
+ */
+public class Context {
+  /** Values of incoming columns from all inputs. */
+  public Object[] values;
+}
+
+// End Context.java

http://git-wip-us.apache.org/repos/asf/incubator-optiq/blob/5fba93bf/core/src/main/java/net/hydromatic/optiq/impl/interpreter/FilterNode.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/net/hydromatic/optiq/impl/interpreter/FilterNode.java b/core/src/main/java/net/hydromatic/optiq/impl/interpreter/FilterNode.java
new file mode 100644
index 0000000..003952c
--- /dev/null
+++ b/core/src/main/java/net/hydromatic/optiq/impl/interpreter/FilterNode.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 net.hydromatic.optiq.impl.interpreter;
+
+import org.eigenbase.rel.FilterRelBase;
+
+/**
+ * Interpreter node that implements a {@link org.eigenbase.rel.FilterRelBase}.
+ */
+public class FilterNode implements Node {
+  private final Scalar condition;
+  private final Source source;
+  private final Sink sink;
+  private final Context context;
+
+  public FilterNode(Interpreter interpreter, FilterRelBase rel) {
+    this.condition = interpreter.compile(rel.getCondition());
+    this.source = interpreter.source(rel, 0);
+    this.sink = interpreter.sink(rel);
+    this.context = interpreter.createContext();
+  }
+
+  public void run() throws InterruptedException {
+    Row row;
+    while ((row = source.receive()) != null) {
+      context.values = row.getValues();
+      Boolean b = (Boolean) condition.execute(context);
+      if (b != null && b) {
+        sink.send(row);
+      }
+    }
+  }
+}
+
+// End FilterNode.java

http://git-wip-us.apache.org/repos/asf/incubator-optiq/blob/5fba93bf/core/src/main/java/net/hydromatic/optiq/impl/interpreter/Interpreter.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/net/hydromatic/optiq/impl/interpreter/Interpreter.java b/core/src/main/java/net/hydromatic/optiq/impl/interpreter/Interpreter.java
new file mode 100644
index 0000000..12ac66b
--- /dev/null
+++ b/core/src/main/java/net/hydromatic/optiq/impl/interpreter/Interpreter.java
@@ -0,0 +1,269 @@
+/*
+ * 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 net.hydromatic.optiq.impl.interpreter;
+
+import net.hydromatic.linq4j.AbstractEnumerable;
+import net.hydromatic.linq4j.Enumerator;
+
+import net.hydromatic.optiq.DataContext;
+
+import org.eigenbase.rel.*;
+import org.eigenbase.rex.*;
+import org.eigenbase.util.ReflectUtil;
+import org.eigenbase.util.ReflectiveVisitDispatcher;
+import org.eigenbase.util.ReflectiveVisitor;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Maps;
+
+import java.util.*;
+
+/**
+ * Interpreter.
+ *
+ * <p>Contains the context for interpreting relational expressions. In
+ * particular it holds working state while the data flow graph is being
+ * assembled.</p>
+ */
+public class Interpreter extends AbstractEnumerable<Row> {
+  final Map<RelNode, NodeInfo> nodes = Maps.newLinkedHashMap();
+  private final DataContext dataContext;
+  private final RelNode rootRel;
+
+  public Interpreter(DataContext dataContext, RelNode rootRel) {
+    this.dataContext = dataContext;
+    this.rootRel = rootRel;
+    Compiler compiler = new Nodes.CoreCompiler(this);
+    compiler.visit(rootRel, 0, null);
+  }
+
+  public Enumerator<Row> enumerator() {
+    start();
+    final ArrayDeque<Row> queue = nodes.get(rootRel).sink.list;
+    return new Enumerator<Row>() {
+      Row row;
+
+      public Row current() {
+        return row;
+      }
+
+      public boolean moveNext() {
+        try {
+          row = queue.removeFirst();
+        } catch (NoSuchElementException e) {
+          return false;
+        }
+        return true;
+      }
+
+      public void reset() {
+        row = null;
+      }
+
+      public void close() {
+        Interpreter.this.close();
+      }
+    };
+  }
+
+  private void start() {
+    // We rely on the nodes being ordered leaves first.
+    for (Map.Entry<RelNode, NodeInfo> entry : nodes.entrySet()) {
+      final NodeInfo nodeInfo = entry.getValue();
+      try {
+        nodeInfo.node.run();
+      } catch (InterruptedException e) {
+        e.printStackTrace();
+      }
+    }
+  }
+
+  private void close() {
+    // TODO:
+  }
+
+  /** Compiles an expression to an executable form. */
+  public Scalar compile(final RexNode node) {
+    if (node instanceof RexCall) {
+      final RexCall call = (RexCall) node;
+      final ImmutableList.Builder<Scalar> list = ImmutableList.builder();
+      for (RexNode operand : call.getOperands()) {
+        list.add(compile(operand));
+      }
+      final ImmutableList<Scalar> scalars = list.build();
+      return new Scalar() {
+        public Object execute(final Context context) {
+          final List<Object> args;
+          final Comparable o0;
+          final Comparable o1;
+          switch (call.getKind()) {
+          case LESS_THAN:
+            args = lazyArgs(context);
+            o0 = (Comparable) args.get(0);
+            o1 = (Comparable) args.get(1);
+            return o0 == null || o1 == null ? null : o0.compareTo(o1) < 0;
+          case GREATER_THAN:
+            args = lazyArgs(context);
+            o0 = (Comparable) args.get(0);
+            o1 = (Comparable) args.get(1);
+            return o0 == null || o1 == null ? null : o0.compareTo(o1) > 0;
+          default:
+            throw new AssertionError("unknown expression " + call);
+          }
+        }
+
+        private List<Object> lazyArgs(final Context context) {
+          return new AbstractList<Object>() {
+            @Override public Object get(int index) {
+              return scalars.get(index).execute(context);
+            }
+
+            @Override public int size() {
+              return scalars.size();
+            }
+          };
+        }
+      };
+    }
+    return new Scalar() {
+      public Object execute(Context context) {
+        switch (node.getKind()) {
+        case LITERAL:
+          return ((RexLiteral) node).getValue();
+        case INPUT_REF:
+          return context.values[((RexInputRef) node).getIndex()];
+        default:
+          throw new RuntimeException("unknown expression type " + node);
+        }
+      }
+    };
+  }
+
+  public Source source(RelNode rel, int ordinal) {
+    final RelNode input = rel.getInput(ordinal);
+    final NodeInfo x = nodes.get(input);
+    if (x == null) {
+      throw new AssertionError("should be registered: " + rel);
+    }
+    return new ListSource(x.sink);
+  }
+
+  public Sink sink(RelNode rel) {
+    final ArrayDeque<Row> queue = new ArrayDeque<Row>(1);
+    final ListSink sink = new ListSink(queue);
+    final NodeInfo nodeInfo = new NodeInfo(rel, sink);
+    nodes.put(rel, nodeInfo);
+    return sink;
+  }
+
+  public Context createContext() {
+    return new Context();
+  }
+
+  public DataContext getDataContext() {
+    return dataContext;
+  }
+
+  /** Information about a node registered in the data flow graph. */
+  private static class NodeInfo {
+    final RelNode rel;
+    final ListSink sink;
+    Node node;
+
+    public NodeInfo(RelNode rel, ListSink sink) {
+      this.rel = rel;
+      this.sink = sink;
+    }
+  }
+
+  /** Implementation of {@link Sink} using a {@link java.util.ArrayDeque}. */
+  private static class ListSink implements Sink {
+    final ArrayDeque<Row> list;
+
+    private ListSink(ArrayDeque<Row> list) {
+      this.list = list;
+    }
+
+    public void send(Row row) throws InterruptedException {
+      list.add(row);
+    }
+
+    public void end() throws InterruptedException {
+    }
+  }
+
+  /** Implementation of {@link Source} using a {@link java.util.ArrayDeque}. */
+  private static class ListSource implements Source {
+    private final ArrayDeque<Row> list;
+
+    public ListSource(ListSink sink) {
+      this.list = sink.list;
+    }
+
+    public Row receive() {
+      try {
+        return list.remove();
+      } catch (NoSuchElementException e) {
+        return null;
+      }
+    }
+  }
+
+  /**
+   * Walks over a tree of {@link org.eigenbase.rel.RelNode} and, for each,
+   * creates a {@link net.hydromatic.optiq.impl.interpreter.Node} that can be
+   * executed in the interpreter.
+   *
+   * <p>The compiler looks for methods of the form "visit(XxxRel)".
+   * A "visit" method must create an appropriate {@link Node} and put it into
+   * the {@link #node} field.
+   *
+   * <p>If you wish to handle more kinds of relational expressions, add extra
+   * "visit" methods in this or a sub-class, and they will be found and called
+   * via reflection.
+   */
+  public static class Compiler extends RelVisitor implements ReflectiveVisitor {
+    private final ReflectiveVisitDispatcher<Compiler, RelNode> dispatcher =
+        ReflectUtil.createDispatcher(Compiler.class, RelNode.class);
+    protected final Interpreter interpreter;
+    protected Node node;
+
+    private static final String VISIT_METHOD_NAME = "visit";
+
+    Compiler(Interpreter interpreter) {
+      this.interpreter = interpreter;
+    }
+
+    @Override public void visit(RelNode p, int ordinal, RelNode parent) {
+      // rewrite children first (from left to right)
+      super.visit(p, ordinal, parent);
+
+      node = null;
+      boolean found = dispatcher.invokeVisitor(this, p, VISIT_METHOD_NAME);
+      if (!found) {
+        // Probably need to add a visit(XxxRel) method to CoreCompiler.
+        throw new AssertionError("interpreter: no implementation for "
+            + p.getClass());
+      }
+      final NodeInfo nodeInfo = interpreter.nodes.get(p);
+      assert nodeInfo != null;
+      nodeInfo.node = node;
+    }
+  }
+}
+
+// End Interpreter.java

http://git-wip-us.apache.org/repos/asf/incubator-optiq/blob/5fba93bf/core/src/main/java/net/hydromatic/optiq/impl/interpreter/Node.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/net/hydromatic/optiq/impl/interpreter/Node.java b/core/src/main/java/net/hydromatic/optiq/impl/interpreter/Node.java
new file mode 100644
index 0000000..8fd0a2d
--- /dev/null
+++ b/core/src/main/java/net/hydromatic/optiq/impl/interpreter/Node.java
@@ -0,0 +1,26 @@
+/*
+ * 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 net.hydromatic.optiq.impl.interpreter;
+
+/**
+ * Relational expression that can be executed using an interpreter.
+ */
+public interface Node {
+  void run() throws InterruptedException;
+}
+
+// End Node.java

http://git-wip-us.apache.org/repos/asf/incubator-optiq/blob/5fba93bf/core/src/main/java/net/hydromatic/optiq/impl/interpreter/Nodes.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/net/hydromatic/optiq/impl/interpreter/Nodes.java b/core/src/main/java/net/hydromatic/optiq/impl/interpreter/Nodes.java
new file mode 100644
index 0000000..c2de888
--- /dev/null
+++ b/core/src/main/java/net/hydromatic/optiq/impl/interpreter/Nodes.java
@@ -0,0 +1,57 @@
+/*
+ * 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 net.hydromatic.optiq.impl.interpreter;
+
+import org.eigenbase.rel.*;
+
+/**
+ * Helper methods for {@link Node} and implementations for core relational
+ * expressions.
+ */
+public class Nodes {
+  /** Extension to
+   * {@link net.hydromatic.optiq.impl.interpreter.Interpreter.Compiler}
+   * that knows how to handle the core logical
+   * {@link org.eigenbase.rel.RelNode}s. */
+  public static class CoreCompiler extends Interpreter.Compiler {
+    CoreCompiler(Interpreter interpreter) {
+      super(interpreter);
+    }
+
+    public void visit(FilterRelBase filter) {
+      node = new FilterNode(interpreter, filter);
+    }
+
+    public void visit(ProjectRelBase project) {
+      node = new ProjectNode(interpreter, project);
+    }
+
+    public void visit(ValuesRelBase value) {
+      node = new ValuesNode(interpreter, value);
+    }
+
+    public void visit(TableAccessRelBase scan) {
+      node = new ScanNode(interpreter, scan);
+    }
+
+    public void visit(SortRel sort) {
+      node = new SortNode(interpreter, sort);
+    }
+  }
+}
+
+// End Node.java

http://git-wip-us.apache.org/repos/asf/incubator-optiq/blob/5fba93bf/core/src/main/java/net/hydromatic/optiq/impl/interpreter/ProjectNode.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/net/hydromatic/optiq/impl/interpreter/ProjectNode.java b/core/src/main/java/net/hydromatic/optiq/impl/interpreter/ProjectNode.java
new file mode 100644
index 0000000..7193057
--- /dev/null
+++ b/core/src/main/java/net/hydromatic/optiq/impl/interpreter/ProjectNode.java
@@ -0,0 +1,58 @@
+/*
+ * 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 net.hydromatic.optiq.impl.interpreter;
+
+import org.eigenbase.rel.ProjectRelBase;
+import org.eigenbase.rex.RexNode;
+
+import com.google.common.collect.ImmutableList;
+
+/**
+ * Interpreter node that implements a {@link org.eigenbase.rel.FilterRel}.
+ */
+public class ProjectNode implements Node {
+  private final ImmutableList<Scalar> projects;
+  private final Source source;
+  private final Sink sink;
+  private final Context context;
+
+  public ProjectNode(Interpreter interpreter, ProjectRelBase rel) {
+    ImmutableList.Builder<Scalar> builder = ImmutableList.builder();
+    for (RexNode node : rel.getProjects()) {
+      builder.add(interpreter.compile(node));
+    }
+    this.projects = builder.build();
+    this.source = interpreter.source(rel, 0);
+    this.sink = interpreter.sink(rel);
+    this.context = interpreter.createContext();
+  }
+
+  public void run() throws InterruptedException {
+    Row row;
+    while ((row = source.receive()) != null) {
+      context.values = row.getValues();
+      Object[] values = new Object[projects.size()];
+      for (int i = 0; i < projects.size(); i++) {
+        Scalar scalar = projects.get(i);
+        values[i] = scalar.execute(context);
+      }
+      sink.send(new Row(values));
+    }
+  }
+}
+
+// End FilterNode.java

http://git-wip-us.apache.org/repos/asf/incubator-optiq/blob/5fba93bf/core/src/main/java/net/hydromatic/optiq/impl/interpreter/Row.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/net/hydromatic/optiq/impl/interpreter/Row.java b/core/src/main/java/net/hydromatic/optiq/impl/interpreter/Row.java
new file mode 100644
index 0000000..e410704
--- /dev/null
+++ b/core/src/main/java/net/hydromatic/optiq/impl/interpreter/Row.java
@@ -0,0 +1,55 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package net.hydromatic.optiq.impl.interpreter;
+
+import java.util.Arrays;
+
+/**
+ * Row.
+ */
+public class Row {
+  private final Object[] values;
+
+  public Row(Object[] values) {
+    this.values = values;
+  }
+
+  @Override public int hashCode() {
+    return Arrays.hashCode(values);
+  }
+
+  @Override public boolean equals(Object obj) {
+    return obj == this
+        || obj instanceof Row
+        && Arrays.equals(values, ((Row) obj).values);
+  }
+
+  @Override public String toString() {
+    return Arrays.toString(values);
+  }
+
+  public Object getObject(int index) {
+    return values[index];
+  }
+
+  // must stay package-protected
+  Object[] getValues() {
+    return values;
+  }
+}
+
+// End Row.java

http://git-wip-us.apache.org/repos/asf/incubator-optiq/blob/5fba93bf/core/src/main/java/net/hydromatic/optiq/impl/interpreter/Scalar.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/net/hydromatic/optiq/impl/interpreter/Scalar.java b/core/src/main/java/net/hydromatic/optiq/impl/interpreter/Scalar.java
new file mode 100644
index 0000000..b3e2118
--- /dev/null
+++ b/core/src/main/java/net/hydromatic/optiq/impl/interpreter/Scalar.java
@@ -0,0 +1,26 @@
+/*
+ * 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 net.hydromatic.optiq.impl.interpreter;
+
+/**
+ * Compiled scalar expression.
+ */
+public interface Scalar {
+  Object execute(Context context);
+}
+
+// End Scalar.java

http://git-wip-us.apache.org/repos/asf/incubator-optiq/blob/5fba93bf/core/src/main/java/net/hydromatic/optiq/impl/interpreter/ScanNode.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/net/hydromatic/optiq/impl/interpreter/ScanNode.java b/core/src/main/java/net/hydromatic/optiq/impl/interpreter/ScanNode.java
new file mode 100644
index 0000000..085ffce
--- /dev/null
+++ b/core/src/main/java/net/hydromatic/optiq/impl/interpreter/ScanNode.java
@@ -0,0 +1,116 @@
+/*
+ * 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 net.hydromatic.optiq.impl.interpreter;
+
+import net.hydromatic.linq4j.Enumerable;
+import net.hydromatic.linq4j.Enumerator;
+import net.hydromatic.linq4j.Queryable;
+import net.hydromatic.linq4j.function.Function1;
+
+import net.hydromatic.optiq.DataContext;
+import net.hydromatic.optiq.QueryableTable;
+import net.hydromatic.optiq.SchemaPlus;
+import net.hydromatic.optiq.Schemas;
+
+import org.eigenbase.rel.TableAccessRelBase;
+import org.eigenbase.util.Util;
+
+import com.google.common.collect.ImmutableList;
+
+import java.lang.reflect.Field;
+import java.lang.reflect.Modifier;
+import java.lang.reflect.Type;
+import java.util.List;
+
+/**
+ * Interpreter node that implements a
+ * {@link org.eigenbase.rel.TableAccessRelBase}.
+ */
+public class ScanNode implements Node {
+  private final Sink sink;
+  private final TableAccessRelBase rel;
+  private final DataContext root;
+
+  public ScanNode(Interpreter interpreter, TableAccessRelBase rel) {
+    this.rel = rel;
+    this.sink = interpreter.sink(rel);
+    this.root = interpreter.getDataContext();
+  }
+
+  public void run() throws InterruptedException {
+    final Enumerable<Row> iterable = iterable();
+    final Enumerator<Row> enumerator = iterable.enumerator();
+    while (enumerator.moveNext()) {
+      sink.send(enumerator.current());
+    }
+    enumerator.close();
+    sink.end();
+  }
+
+  private Enumerable<Row> iterable() {
+    //noinspection unchecked
+    Enumerable<Row> iterable = rel.getTable().unwrap(Enumerable.class);
+    if (iterable != null) {
+      return iterable;
+    }
+    final QueryableTable queryableTable =
+        rel.getTable().unwrap(QueryableTable.class);
+    if (queryableTable != null) {
+      final Type elementType = queryableTable.getElementType();
+      SchemaPlus schema = root.getRootSchema();
+      for (String name : Util.skipLast(rel.getTable().getQualifiedName())) {
+        schema = schema.getSubSchema(name);
+      }
+      if (elementType instanceof Class) {
+        //noinspection unchecked
+        final Queryable<Object> queryable = Schemas.queryable(root,
+            (Class) elementType, rel.getTable().getQualifiedName());
+        ImmutableList.Builder<Field> fieldBuilder = ImmutableList.builder();
+        Class type = (Class) elementType;
+        for (Field field : type.getFields()) {
+          if (Modifier.isPublic(field.getModifiers())
+              && !Modifier.isStatic(field.getModifiers())) {
+            fieldBuilder.add(field);
+          }
+        }
+        final List<Field> fields = fieldBuilder.build();
+        return queryable.select(
+            new Function1<Object, Row>() {
+              public Row apply(Object o) {
+                final Object[] values = new Object[fields.size()];
+                for (int i = 0; i < fields.size(); i++) {
+                  Field field = fields.get(i);
+                  try {
+                    values[i] = field.get(o);
+                  } catch (IllegalAccessException e) {
+                    throw new RuntimeException(e);
+                  }
+                }
+                return new Row(values);
+              }
+            });
+      } else {
+        return Schemas.queryable(root, Row.class,
+            rel.getTable().getQualifiedName());
+      }
+    }
+    throw new AssertionError("cannot convert table " + rel.getTable()
+        + " to iterable");
+  }
+}
+
+// End ScanNode.java

http://git-wip-us.apache.org/repos/asf/incubator-optiq/blob/5fba93bf/core/src/main/java/net/hydromatic/optiq/impl/interpreter/Sink.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/net/hydromatic/optiq/impl/interpreter/Sink.java b/core/src/main/java/net/hydromatic/optiq/impl/interpreter/Sink.java
new file mode 100644
index 0000000..6a07c1d
--- /dev/null
+++ b/core/src/main/java/net/hydromatic/optiq/impl/interpreter/Sink.java
@@ -0,0 +1,30 @@
+/*
+ * 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 net.hydromatic.optiq.impl.interpreter;
+
+/**
+ * Sink to which to send rows.
+ *
+ * <p>Corresponds to an output of a relational expression.
+ */
+public interface Sink {
+  void send(Row row) throws InterruptedException;
+
+  void end() throws InterruptedException;
+}
+
+// End Sink.java

http://git-wip-us.apache.org/repos/asf/incubator-optiq/blob/5fba93bf/core/src/main/java/net/hydromatic/optiq/impl/interpreter/SortNode.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/net/hydromatic/optiq/impl/interpreter/SortNode.java b/core/src/main/java/net/hydromatic/optiq/impl/interpreter/SortNode.java
new file mode 100644
index 0000000..1f324c3
--- /dev/null
+++ b/core/src/main/java/net/hydromatic/optiq/impl/interpreter/SortNode.java
@@ -0,0 +1,131 @@
+/*
+ * 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 net.hydromatic.optiq.impl.interpreter;
+
+import org.eigenbase.rel.RelFieldCollation;
+import org.eigenbase.rel.SortRel;
+import org.eigenbase.rex.RexLiteral;
+
+import com.google.common.base.Function;
+import com.google.common.collect.Iterables;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Ordering;
+
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+
+/**
+ * Interpreter node that implements a
+ * {@link org.eigenbase.rel.SortRel}.
+ */
+public class SortNode implements Node {
+  private final Source source;
+  private final Sink sink;
+  private final SortRel rel;
+
+  public SortNode(Interpreter interpreter, SortRel rel) {
+    this.rel = rel;
+    this.source = interpreter.source(rel, 0);
+    this.sink = interpreter.sink(rel);
+  }
+
+  public void run() throws InterruptedException {
+    final int offset =
+        rel.offset == null
+            ? 0
+            : (Integer) ((RexLiteral) rel.offset).getValue();
+    final int fetch =
+        rel.fetch == null
+            ? -1
+            : (Integer) ((RexLiteral) rel.fetch).getValue();
+    // In pure limit mode. No sort required.
+    Row row;
+  loop:
+    if (rel.getCollation().getFieldCollations().isEmpty()) {
+      for (int i = 0; i < offset; i++) {
+        row = source.receive();
+        if (row == null) {
+          break loop;
+        }
+      }
+      if (fetch >= 0) {
+        for (int i = 0; i < offset && (row = source.receive()) != null; i++) {
+          sink.send(row);
+        }
+      } else {
+        while ((row = source.receive()) != null) {
+          sink.send(row);
+        }
+      }
+    } else {
+      // Build a sorted collection.
+      final List<Row> list = Lists.newArrayList();
+      while ((row = source.receive()) != null) {
+        list.add(row);
+      }
+      Collections.sort(list, comparator());
+      final int end = fetch < 0 || offset + fetch > list.size()
+          ? list.size()
+          : offset + fetch;
+      for (int i = offset; i < end; i++) {
+        sink.send(list.get(i));
+      }
+    }
+    sink.end();
+  }
+
+  private Comparator<Row> comparator() {
+    if (rel.getCollation().getFieldCollations().size() == 1) {
+      return comparator(rel.getCollation().getFieldCollations().get(0));
+    }
+    return Ordering.compound(
+        Iterables.transform(rel.getCollation().getFieldCollations(),
+            new Function<RelFieldCollation, Comparator<? super Row>>() {
+              public Comparator<? super Row> apply(RelFieldCollation input) {
+                return comparator(input);
+              }
+            }));
+  }
+
+  private Comparator<Row> comparator(final RelFieldCollation fieldCollation) {
+    switch (fieldCollation.direction) {
+    case ASCENDING:
+      return new Comparator<Row>() {
+        final int x = fieldCollation.getFieldIndex();
+        public int compare(Row o1, Row o2) {
+          final Comparable c1 = (Comparable) o1.getValues()[x];
+          final Comparable c2 = (Comparable) o2.getValues()[x];
+          //noinspection unchecked
+          return c1.compareTo(c2);
+        }
+      };
+    default:
+      return new Comparator<Row>() {
+        final int x = fieldCollation.getFieldIndex();
+        public int compare(Row o1, Row o2) {
+          final Comparable c1 = (Comparable) o1.getValues()[x];
+          final Comparable c2 = (Comparable) o2.getValues()[x];
+          //noinspection unchecked
+          return c2.compareTo(c1);
+        }
+      };
+    }
+  }
+}
+
+// End ScanNode.java

http://git-wip-us.apache.org/repos/asf/incubator-optiq/blob/5fba93bf/core/src/main/java/net/hydromatic/optiq/impl/interpreter/Source.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/net/hydromatic/optiq/impl/interpreter/Source.java b/core/src/main/java/net/hydromatic/optiq/impl/interpreter/Source.java
new file mode 100644
index 0000000..6f67d32
--- /dev/null
+++ b/core/src/main/java/net/hydromatic/optiq/impl/interpreter/Source.java
@@ -0,0 +1,29 @@
+/*
+ * 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 net.hydromatic.optiq.impl.interpreter;
+
+/**
+ * Source of rows.
+ *
+ * <p>Corresponds to an input of a relational expression.
+ */
+public interface Source {
+  /** Reads a row. Null means end of data. */
+  Row receive();
+}
+
+// End Source.java

http://git-wip-us.apache.org/repos/asf/incubator-optiq/blob/5fba93bf/core/src/main/java/net/hydromatic/optiq/impl/interpreter/ValuesNode.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/net/hydromatic/optiq/impl/interpreter/ValuesNode.java b/core/src/main/java/net/hydromatic/optiq/impl/interpreter/ValuesNode.java
new file mode 100644
index 0000000..e60e797
--- /dev/null
+++ b/core/src/main/java/net/hydromatic/optiq/impl/interpreter/ValuesNode.java
@@ -0,0 +1,51 @@
+/*
+ * 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 net.hydromatic.optiq.impl.interpreter;
+
+import org.eigenbase.rel.ValuesRelBase;
+import org.eigenbase.rex.RexLiteral;
+
+import java.util.List;
+
+/**
+ * Interpreter node that implements a {@link ValuesRelBase}.
+ */
+public class ValuesNode implements Node {
+  private final Sink sink;
+  private final ValuesRelBase rel;
+  private final int fieldCount;
+
+  public ValuesNode(Interpreter interpreter, ValuesRelBase rel) {
+    this.rel = rel;
+    this.sink = interpreter.sink(rel);
+    this.fieldCount = rel.getRowType().getFieldCount();
+  }
+
+  public void run() throws InterruptedException {
+    for (List<RexLiteral> list : rel.getTuples()) {
+      final Object[] values = new Object[fieldCount];
+      for (int i = 0; i < list.size(); i++) {
+        RexLiteral rexLiteral = list.get(i);
+        values[i] = rexLiteral.getValue();
+      }
+      sink.send(new Row(values));
+    }
+    sink.end();
+  }
+}
+
+// End ValuesNode.java

http://git-wip-us.apache.org/repos/asf/incubator-optiq/blob/5fba93bf/core/src/main/java/net/hydromatic/optiq/impl/interpreter/package-info.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/net/hydromatic/optiq/impl/interpreter/package-info.java b/core/src/main/java/net/hydromatic/optiq/impl/interpreter/package-info.java
new file mode 100644
index 0000000..712d56e
--- /dev/null
+++ b/core/src/main/java/net/hydromatic/optiq/impl/interpreter/package-info.java
@@ -0,0 +1,27 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+/**
+ * Provides an implementation of relational expressions using an interpreter.
+ *
+ * <p>The implementation is not efficient compared to generated code, but
+ * preparation time is less, and so the total prepare + execute time is
+ * competitive for queries over small data sets.
+ */
+package net.hydromatic.optiq.impl.interpreter;
+
+// End package-info.java

http://git-wip-us.apache.org/repos/asf/incubator-optiq/blob/5fba93bf/core/src/main/java/net/hydromatic/optiq/jdbc/JavaTypeFactoryImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/net/hydromatic/optiq/jdbc/JavaTypeFactoryImpl.java b/core/src/main/java/net/hydromatic/optiq/jdbc/JavaTypeFactoryImpl.java
index 67fb093..6d173f6 100644
--- a/core/src/main/java/net/hydromatic/optiq/jdbc/JavaTypeFactoryImpl.java
+++ b/core/src/main/java/net/hydromatic/optiq/jdbc/JavaTypeFactoryImpl.java
@@ -64,7 +64,7 @@ public class JavaTypeFactoryImpl
   public RelDataType createStructType(Class type) {
     List<RelDataTypeField> list = new ArrayList<RelDataTypeField>();
     for (Field field : type.getFields()) {
-      if ((field.getModifiers() & Modifier.STATIC) == 0) {
+      if (!Modifier.isStatic(field.getModifiers())) {
         // FIXME: watch out for recursion
         list.add(
             new RelDataTypeFieldImpl(

http://git-wip-us.apache.org/repos/asf/incubator-optiq/blob/5fba93bf/core/src/test/java/net/hydromatic/optiq/test/InterpreterTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/net/hydromatic/optiq/test/InterpreterTest.java b/core/src/test/java/net/hydromatic/optiq/test/InterpreterTest.java
new file mode 100644
index 0000000..7cf16b6
--- /dev/null
+++ b/core/src/test/java/net/hydromatic/optiq/test/InterpreterTest.java
@@ -0,0 +1,136 @@
+/*
+ * 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 net.hydromatic.optiq.test;
+
+import net.hydromatic.linq4j.QueryProvider;
+
+import net.hydromatic.optiq.DataContext;
+import net.hydromatic.optiq.SchemaPlus;
+import net.hydromatic.optiq.config.Lex;
+import net.hydromatic.optiq.impl.interpreter.Interpreter;
+import net.hydromatic.optiq.impl.interpreter.Row;
+import net.hydromatic.optiq.impl.java.JavaTypeFactory;
+import net.hydromatic.optiq.tools.FrameworkConfig;
+import net.hydromatic.optiq.tools.Frameworks;
+import net.hydromatic.optiq.tools.Planner;
+
+import org.eigenbase.rel.RelNode;
+import org.eigenbase.sql.SqlNode;
+
+import com.google.common.collect.Lists;
+
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.util.Arrays;
+import java.util.List;
+
+import static org.hamcrest.CoreMatchers.equalTo;
+import static org.junit.Assert.assertThat;
+
+/**
+ * Unit tests for {@link net.hydromatic.optiq.impl.interpreter.Interpreter}.
+ */
+public class InterpreterTest {
+  private SchemaPlus rootSchema;
+  private Planner planner;
+
+  /** Implementation of {@link DataContext} for executing queries without a
+   * connection. */
+  private class MyDataContext implements DataContext {
+    private final Planner planner;
+
+    public MyDataContext(Planner planner) {
+      this.planner = planner;
+    }
+
+    public SchemaPlus getRootSchema() {
+      return rootSchema;
+    }
+
+    public JavaTypeFactory getTypeFactory() {
+      return (JavaTypeFactory) planner.getTypeFactory();
+    }
+
+    public QueryProvider getQueryProvider() {
+      return null;
+    }
+
+    public Object get(String name) {
+      return null;
+    }
+  }
+
+  @Before public void setUp() {
+    rootSchema = Frameworks.createRootSchema(true);
+    final FrameworkConfig config = Frameworks.newConfigBuilder()
+        .lex(Lex.ORACLE)
+        .defaultSchema(
+            OptiqAssert.addSchema(rootSchema, OptiqAssert.SchemaSpec.HR))
+        .build();
+    planner = Frameworks.getPlanner(config);
+  }
+
+  @After public void tearDown() {
+    rootSchema = null;
+    planner = null;
+  }
+
+  /** Tests executing a simple plan using an interpreter. */
+  @Test public void testInterpretProjectFilterValues() throws Exception {
+    SqlNode parse =
+        planner.parse("select y, x\n"
+            + "from (values (1, 'a'), (2, 'b'), (3, 'c')) as t(x, y)\n"
+            + "where x > 1");
+
+    SqlNode validate = planner.validate(parse);
+    RelNode convert = planner.convert(validate);
+
+    final Interpreter interpreter = new Interpreter(null, convert);
+    assertRows(interpreter,
+        "[_ISO-8859-1'b', 2]",
+        "[_ISO-8859-1'c', 3]");
+  }
+
+  private static void assertRows(Interpreter interpreter, String... rows) {
+    final List<String> list = Lists.newArrayList();
+    for (Row row : interpreter) {
+      list.add(row.toString());
+    }
+    assertThat(list, equalTo(Arrays.asList(rows)));
+  }
+
+  /** Tests executing a simple plan using an interpreter. */
+  @Test public void testInterpretTable() throws Exception {
+    SqlNode parse =
+        planner.parse("select * from \"hr\".\"emps\" order by \"empid\"");
+
+    SqlNode validate = planner.validate(parse);
+    RelNode convert = planner.convert(validate);
+
+    final Interpreter interpreter =
+        new Interpreter(new MyDataContext(planner), convert);
+    assertRows(interpreter,
+        "[100, 10, Bill, 10000.0, 1000]",
+        "[110, 10, Theodore, 11500.0, 250]",
+        "[150, 10, Sebastian, 7000.0, null]",
+        "[200, 20, Eric, 8000.0, 500]");
+  }
+}
+
+// End InterpreterTest.java

http://git-wip-us.apache.org/repos/asf/incubator-optiq/blob/5fba93bf/core/src/test/java/net/hydromatic/optiq/test/OptiqSuite.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/net/hydromatic/optiq/test/OptiqSuite.java b/core/src/test/java/net/hydromatic/optiq/test/OptiqSuite.java
index 7c3702f..6e72773 100644
--- a/core/src/test/java/net/hydromatic/optiq/test/OptiqSuite.java
+++ b/core/src/test/java/net/hydromatic/optiq/test/OptiqSuite.java
@@ -67,6 +67,7 @@ import org.junit.runners.Suite;
     ModelTest.class,
     SqlValidatorFeatureTest.class,
     VolcanoPlannerTraitTest.class,
+    InterpreterTest.class,
     VolcanoPlannerTest.class,
     SargTest.class,
     SqlPrettyWriterTest.class,