You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@phoenix.apache.org by ja...@apache.org on 2014/01/27 20:23:19 UTC

[18/51] [partial] Initial commit

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/c5b80246/src/main/java/org/apache/phoenix/parse/ParseNodeRewriter.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/phoenix/parse/ParseNodeRewriter.java b/src/main/java/org/apache/phoenix/parse/ParseNodeRewriter.java
new file mode 100644
index 0000000..ffd9adb
--- /dev/null
+++ b/src/main/java/org/apache/phoenix/parse/ParseNodeRewriter.java
@@ -0,0 +1,504 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.phoenix.parse;
+
+import java.sql.SQLException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
+
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import org.apache.phoenix.compile.ColumnResolver;
+import org.apache.phoenix.schema.AmbiguousColumnException;
+import org.apache.phoenix.schema.ColumnNotFoundException;
+
+/**
+ * 
+ * Base class for visitors that rewrite the expression node hierarchy
+ *
+ * @author jtaylor 
+ * @since 0.1
+ */
+public class ParseNodeRewriter extends TraverseAllParseNodeVisitor<ParseNode> {
+    
+    protected static final ParseNodeFactory NODE_FACTORY = new ParseNodeFactory();
+
+    
+    /**
+     * Rewrite the select statement by switching any constants to the right hand side
+     * of the expression.
+     * @param statement the select statement
+     * @return new select statement
+     * @throws SQLException 
+     */
+    public static SelectStatement rewrite(SelectStatement statement, ParseNodeRewriter rewriter) throws SQLException {
+        Map<String,ParseNode> aliasMap = rewriter.getAliasMap();
+        ParseNode where = statement.getWhere();
+        ParseNode normWhere = where;
+        if (where != null) {
+            rewriter.reset();
+            normWhere = where.accept(rewriter);
+        }
+        ParseNode having = statement.getHaving();
+        ParseNode normHaving= having;
+        if (having != null) {
+            rewriter.reset();
+            normHaving = having.accept(rewriter);
+        }
+        List<AliasedNode> selectNodes = statement.getSelect();
+        List<AliasedNode> normSelectNodes = selectNodes;
+        for (int i = 0; i < selectNodes.size(); i++) {
+            AliasedNode aliasedNode = selectNodes.get(i);
+            ParseNode selectNode = aliasedNode.getNode();
+            rewriter.reset();
+            ParseNode normSelectNode = selectNode.accept(rewriter);
+            if (selectNode == normSelectNode) {
+                if (selectNodes != normSelectNodes) {
+                    normSelectNodes.add(aliasedNode);
+                }
+                continue;
+            }
+            if (selectNodes == normSelectNodes) {
+                normSelectNodes = Lists.newArrayList(selectNodes.subList(0, i));
+            }
+            AliasedNode normAliasedNode = NODE_FACTORY.aliasedNode(aliasedNode.getAlias(), normSelectNode);
+            normSelectNodes.add(normAliasedNode);
+        }
+        // Add to map in separate pass so that we don't try to use aliases
+        // while processing the select expressions
+        if (aliasMap != null) {
+            for (int i = 0; i < normSelectNodes.size(); i++) {
+                AliasedNode aliasedNode = normSelectNodes.get(i);
+                ParseNode selectNode = aliasedNode.getNode();
+                String alias = aliasedNode.getAlias();
+                if (alias != null) {
+                    aliasMap.put(alias, selectNode);
+                }
+            }
+        }
+        
+        List<ParseNode> groupByNodes = statement.getGroupBy();
+        List<ParseNode> normGroupByNodes = groupByNodes;
+        for (int i = 0; i < groupByNodes.size(); i++) {
+            ParseNode groupByNode = groupByNodes.get(i);
+            rewriter.reset();
+            ParseNode normGroupByNode = groupByNode.accept(rewriter);
+            if (groupByNode == normGroupByNode) {
+                if (groupByNodes != normGroupByNodes) {
+                    normGroupByNodes.add(groupByNode);
+                }
+                continue;
+            }
+            if (groupByNodes == normGroupByNodes) {
+                normGroupByNodes = Lists.newArrayList(groupByNodes.subList(0, i));
+            }
+            normGroupByNodes.add(normGroupByNode);
+        }
+        List<OrderByNode> orderByNodes = statement.getOrderBy();
+        List<OrderByNode> normOrderByNodes = orderByNodes;
+        for (int i = 0; i < orderByNodes.size(); i++) {
+            OrderByNode orderByNode = orderByNodes.get(i);
+            ParseNode node = orderByNode.getNode();
+            rewriter.reset();
+            ParseNode normNode = node.accept(rewriter);
+            if (node == normNode) {
+                if (orderByNodes != normOrderByNodes) {
+                    normOrderByNodes.add(orderByNode);
+                }
+                continue;
+            }
+            if (orderByNodes == normOrderByNodes) {
+                normOrderByNodes = Lists.newArrayList(orderByNodes.subList(0, i));
+            }
+            normOrderByNodes.add(NODE_FACTORY.orderBy(normNode, orderByNode.isNullsLast(), orderByNode.isAscending()));
+        }
+        
+        // Return new SELECT statement with updated WHERE clause
+        if (normWhere == where && 
+                normHaving == having && 
+                selectNodes == normSelectNodes && 
+                groupByNodes == normGroupByNodes &&
+                orderByNodes == normOrderByNodes) {
+            return statement;
+        }
+        return NODE_FACTORY.select(statement.getFrom(), statement.getHint(), statement.isDistinct(),
+                normSelectNodes, normWhere, normGroupByNodes, normHaving, normOrderByNodes,
+                statement.getLimit(), statement.getBindCount(), statement.isAggregate());
+    }
+    
+    private Map<String, ParseNode> getAliasMap() {
+        return aliasMap;
+    }
+
+    private final ColumnResolver resolver;
+    private final Map<String, ParseNode> aliasMap;
+    private int nodeCount;
+    
+    public boolean isTopLevel() {
+        return nodeCount == 0;
+    }
+    
+    protected ParseNodeRewriter() {
+        this.resolver = null;
+        this.aliasMap = null;
+    }
+    
+    protected ParseNodeRewriter(ColumnResolver resolver) {
+        this.resolver = resolver;
+        this.aliasMap = null;
+    }
+    
+    protected ParseNodeRewriter(ColumnResolver resolver, int maxAliasCount) {
+        this.resolver = resolver;
+        this.aliasMap = Maps.newHashMapWithExpectedSize(maxAliasCount);
+    }
+    
+    protected ColumnResolver getResolver() {
+        return resolver;
+    }
+    
+    protected void reset() {
+        this.nodeCount = 0;
+    }
+    
+    private static interface CompoundNodeFactory {
+        ParseNode createNode(List<ParseNode> children);
+    }
+    
+    private ParseNode leaveCompoundNode(CompoundParseNode node, List<ParseNode> children, CompoundNodeFactory factory) {
+        if (children.equals(node.getChildren())) {
+            return node;
+        } else { // Child nodes have been inverted (because a literal was found on LHS)
+            return factory.createNode(children);
+        }
+    }
+    
+    @Override
+    public ParseNode visitLeave(AndParseNode node, List<ParseNode> nodes) throws SQLException {
+        return leaveCompoundNode(node, nodes, new CompoundNodeFactory() {
+            @Override
+            public ParseNode createNode(List<ParseNode> children) {
+                return NODE_FACTORY.and(children);
+            }
+        });
+    }
+
+    @Override
+    public ParseNode visitLeave(OrParseNode node, List<ParseNode> nodes) throws SQLException {
+        return leaveCompoundNode(node, nodes, new CompoundNodeFactory() {
+            @Override
+            public ParseNode createNode(List<ParseNode> children) {
+                return NODE_FACTORY.or(children);
+            }
+        });
+    }
+    
+    @Override
+    public ParseNode visitLeave(SubtractParseNode node, List<ParseNode> nodes) throws SQLException {
+        return leaveCompoundNode(node, nodes, new CompoundNodeFactory() {
+            @Override
+            public ParseNode createNode(List<ParseNode> children) {
+                return NODE_FACTORY.subtract(children);
+            }
+        });
+    }
+    
+    @Override
+    public ParseNode visitLeave(AddParseNode node, List<ParseNode> nodes) throws SQLException {
+        return leaveCompoundNode(node, nodes, new CompoundNodeFactory() {
+            @Override
+            public ParseNode createNode(List<ParseNode> children) {
+                return NODE_FACTORY.add(children);
+            }
+        });
+    }
+    
+    @Override
+    public ParseNode visitLeave(MultiplyParseNode node, List<ParseNode> nodes) throws SQLException {
+        return leaveCompoundNode(node, nodes, new CompoundNodeFactory() {
+            @Override
+            public ParseNode createNode(List<ParseNode> children) {
+                return NODE_FACTORY.multiply(children);
+            }
+        });
+    }
+    
+    @Override
+    public ParseNode visitLeave(DivideParseNode node, List<ParseNode> nodes) throws SQLException {
+        return leaveCompoundNode(node, nodes, new CompoundNodeFactory() {
+            @Override
+            public ParseNode createNode(List<ParseNode> children) {
+                return NODE_FACTORY.divide(children);
+            }
+        });
+    }
+    
+    @Override
+    public ParseNode visitLeave(final FunctionParseNode node, List<ParseNode> nodes) throws SQLException {
+        return leaveCompoundNode(node, nodes, new CompoundNodeFactory() {
+            @Override
+            public ParseNode createNode(List<ParseNode> children) {
+                return NODE_FACTORY.function(node.getName(),children);
+            }
+        });
+    }
+    
+    @Override
+    public ParseNode visitLeave(CaseParseNode node, List<ParseNode> nodes) throws SQLException {
+        return leaveCompoundNode(node, nodes, new CompoundNodeFactory() {
+            @Override
+            public ParseNode createNode(List<ParseNode> children) {
+                return NODE_FACTORY.caseWhen(children);
+            }
+        });
+    }
+
+    @Override
+    public ParseNode visitLeave(final LikeParseNode node, List<ParseNode> nodes) throws SQLException {
+        return leaveCompoundNode(node, nodes, new CompoundNodeFactory() {
+            @Override
+            public ParseNode createNode(List<ParseNode> children) {
+                return NODE_FACTORY.like(children.get(0),children.get(1),node.isNegate());
+            }
+        });
+    }
+    
+    @Override
+    public ParseNode visitLeave(NotParseNode node, List<ParseNode> nodes) throws SQLException {
+        return leaveCompoundNode(node, nodes, new CompoundNodeFactory() {
+            @Override
+            public ParseNode createNode(List<ParseNode> children) {
+                return NODE_FACTORY.not(children.get(0));
+            }
+        });
+    }
+    
+    @Override
+    public ParseNode visitLeave(final CastParseNode node, List<ParseNode> nodes) throws SQLException {
+        return leaveCompoundNode(node, nodes, new CompoundNodeFactory() {
+            @Override
+            public ParseNode createNode(List<ParseNode> children) {
+                return NODE_FACTORY.cast(children.get(0), node.getDataType());
+            }
+        });
+    }
+    
+    @Override
+    public ParseNode visitLeave(final InListParseNode node, List<ParseNode> nodes) throws SQLException {
+        return leaveCompoundNode(node, nodes, new CompoundNodeFactory() {
+            @Override
+            public ParseNode createNode(List<ParseNode> children) {
+                return NODE_FACTORY.inList(children, node.isNegate());
+            }
+        });
+    }
+    
+    @Override
+    public ParseNode visitLeave(final IsNullParseNode node, List<ParseNode> nodes) throws SQLException {
+        return leaveCompoundNode(node, nodes, new CompoundNodeFactory() {
+            @Override
+            public ParseNode createNode(List<ParseNode> children) {
+                return NODE_FACTORY.isNull(children.get(0), node.isNegate());
+            }
+        });
+    }
+    
+    /**
+     * Rewrites expressions of the form (a, b, c) = (1, 2) as a = 1 and b = 2 and c is null
+     * as this is equivalent and already optimized
+     * @param lhs
+     * @param rhs
+     * @param andNodes
+     * @throws SQLException 
+     */
+    private void rewriteRowValueConstuctorEqualityComparison(ParseNode lhs, ParseNode rhs, List<ParseNode> andNodes) throws SQLException {
+        if (lhs instanceof RowValueConstructorParseNode && rhs instanceof RowValueConstructorParseNode) {
+            int i = 0;
+            for (; i < Math.min(lhs.getChildren().size(),rhs.getChildren().size()); i++) {
+                rewriteRowValueConstuctorEqualityComparison(lhs.getChildren().get(i), rhs.getChildren().get(i), andNodes);
+            }
+            for (; i < lhs.getChildren().size(); i++) {
+                rewriteRowValueConstuctorEqualityComparison(lhs.getChildren().get(i), null, andNodes);
+            }
+            for (; i < rhs.getChildren().size(); i++) {
+                rewriteRowValueConstuctorEqualityComparison(null, rhs.getChildren().get(i), andNodes);
+            }
+        } else if (lhs instanceof RowValueConstructorParseNode) {
+            rewriteRowValueConstuctorEqualityComparison(lhs.getChildren().get(0), rhs, andNodes);
+            for (int i = 1; i < lhs.getChildren().size(); i++) {
+                rewriteRowValueConstuctorEqualityComparison(lhs.getChildren().get(i), null, andNodes);
+            }
+        } else if (rhs instanceof RowValueConstructorParseNode) {
+            rewriteRowValueConstuctorEqualityComparison(lhs, rhs.getChildren().get(0), andNodes);
+            for (int i = 1; i < rhs.getChildren().size(); i++) {
+                rewriteRowValueConstuctorEqualityComparison(null, rhs.getChildren().get(i), andNodes);
+            }
+        } else if (lhs == null && rhs == null) { // null == null will end up making the query degenerate
+            andNodes.add(NODE_FACTORY.comparison(CompareOp.EQUAL, null, null).accept(this));
+        } else if (lhs == null) { // AND rhs IS NULL
+            andNodes.add(NODE_FACTORY.isNull(rhs, false).accept(this));
+        } else if (rhs == null) { // AND lhs IS NULL
+            andNodes.add(NODE_FACTORY.isNull(lhs, false).accept(this));
+        } else { // AND lhs = rhs
+            andNodes.add(NODE_FACTORY.comparison(CompareOp.EQUAL, lhs, rhs).accept(this));
+        }
+    }
+    
+    @Override
+    public ParseNode visitLeave(final ComparisonParseNode node, List<ParseNode> nodes) throws SQLException {
+        ParseNode normNode = leaveCompoundNode(node, nodes, new CompoundNodeFactory() {
+            @Override
+            public ParseNode createNode(List<ParseNode> children) {
+                return NODE_FACTORY.comparison(node.getFilterOp(), children.get(0), children.get(1));
+            }
+        });
+        
+        CompareOp op = node.getFilterOp();
+        if (op == CompareOp.EQUAL || op == CompareOp.NOT_EQUAL) {
+            // Rewrite row value constructor in = or != expression, as this is the same as if it was
+            // used in an equality expression for each individual part.
+            ParseNode lhs = normNode.getChildren().get(0);
+            ParseNode rhs = normNode.getChildren().get(1);
+            if (lhs instanceof RowValueConstructorParseNode || rhs instanceof RowValueConstructorParseNode) {
+                List<ParseNode> andNodes = Lists.newArrayListWithExpectedSize(Math.max(lhs.getChildren().size(), rhs.getChildren().size()));
+                rewriteRowValueConstuctorEqualityComparison(lhs,rhs,andNodes);
+                normNode = NODE_FACTORY.and(andNodes);
+                if (op == CompareOp.NOT_EQUAL) {
+                    normNode = NODE_FACTORY.not(normNode);
+                }
+            }
+        }
+        return normNode;
+    }
+    
+    @Override
+    public ParseNode visitLeave(final BetweenParseNode node, List<ParseNode> nodes) throws SQLException {
+        return leaveCompoundNode(node, nodes, new CompoundNodeFactory() {
+            @Override
+            public ParseNode createNode(List<ParseNode> children) {
+                if(node.isNegate()) {
+                    return NODE_FACTORY.not(NODE_FACTORY.and(children));
+                } else {
+                    return NODE_FACTORY.and(children);
+                }
+            }
+        });
+    }
+    
+    @Override
+    public ParseNode visit(ColumnParseNode node) throws SQLException {
+        // If we're resolving aliases and we have an unqualified ColumnParseNode,
+        // check if we find the name in our alias map.
+        if (aliasMap != null && node.getTableName() == null) {
+            ParseNode aliasedNode = aliasMap.get(node.getName());
+            // If we found something, then try to resolve it unless the two nodes are the same
+            if (aliasedNode != null && !node.equals(aliasedNode)) {
+                try {
+                    // If we're able to resolve it, that means we have a conflict
+                    resolver.resolveColumn(node.getSchemaName(), node.getTableName(), node.getName());
+                    throw new AmbiguousColumnException(node.getName());
+                } catch (ColumnNotFoundException e) {
+                    // Not able to resolve alias as a column name as well, so we use the alias
+                    return aliasedNode;
+                }
+            }
+        }
+        return node;
+    }
+
+    @Override
+    public ParseNode visit(LiteralParseNode node) throws SQLException {
+        return node;
+    }
+
+    @Override
+    public ParseNode visit(BindParseNode node) throws SQLException {
+        return node;
+    }
+    
+    @Override
+    public ParseNode visit(WildcardParseNode node) throws SQLException {
+        return node;
+    }
+    
+    @Override
+    public ParseNode visit(FamilyWildcardParseNode node) throws SQLException {
+        return node;
+    }
+    
+    @Override
+    public List<ParseNode> newElementList(int size) {
+        nodeCount += size;
+        return new ArrayList<ParseNode>(size);
+    }
+    
+    @Override
+    public ParseNode visitLeave(StringConcatParseNode node, List<ParseNode> l) throws SQLException {
+        return node;
+    }
+
+    @Override
+    public void addElement(List<ParseNode> l, ParseNode element) {
+        nodeCount--;
+        if (element != null) {
+            l.add(element);
+        }
+    }
+
+    @Override
+    public ParseNode visitLeave(RowValueConstructorParseNode node, List<ParseNode> children) throws SQLException {
+        // Strip trailing nulls from rvc as they have no meaning
+        if (children.get(children.size()-1) == null) {
+            children = Lists.newArrayList(children);
+            do {
+                children.remove(children.size()-1);
+            } while (children.size() > 0 && children.get(children.size()-1) == null);
+            // If we're down to a single child, it's not a rvc anymore
+            if (children.size() == 0) {
+                return null;
+            }
+            if (children.size() == 1) {
+                return children.get(0);
+            }
+        }
+        // Flatten nested row value constructors, as this makes little sense and adds no information
+        List<ParseNode> flattenedChildren = children;
+        for (int i = 0; i < children.size(); i++) {
+            ParseNode child = children.get(i);
+            if (child instanceof RowValueConstructorParseNode) {
+                if (flattenedChildren == children) {
+                    flattenedChildren = Lists.newArrayListWithExpectedSize(children.size() + child.getChildren().size());
+                    flattenedChildren.addAll(children.subList(0, i));
+                }
+                flattenedChildren.addAll(child.getChildren());
+            }
+        }
+        
+        return leaveCompoundNode(node, flattenedChildren, new CompoundNodeFactory() {
+            @Override
+            public ParseNode createNode(List<ParseNode> children) {
+                return NODE_FACTORY.rowValueConstructor(children);
+            }
+        });
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/c5b80246/src/main/java/org/apache/phoenix/parse/ParseNodeVisitor.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/phoenix/parse/ParseNodeVisitor.java b/src/main/java/org/apache/phoenix/parse/ParseNodeVisitor.java
new file mode 100644
index 0000000..60884bb
--- /dev/null
+++ b/src/main/java/org/apache/phoenix/parse/ParseNodeVisitor.java
@@ -0,0 +1,104 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.phoenix.parse;
+
+import java.sql.SQLException;
+import java.util.List;
+
+
+
+/**
+ * 
+ * Visitor for ParseNode in the node tree. Uses composite
+ * visitor pattern with enter/leave calls for any
+ * compound expression node. Only supported SQL constructs
+ * have visit methods.  Unsupported constructs fall through
+ * to {@link #visitEnter(CompoundParseNode)} for
+ * compound parse nodes and {@link #visit(ParseNode)}
+ * for terminal parse nodes.
+ * 
+ * @author jtaylor
+ * @since 0.1
+ */
+public interface ParseNodeVisitor<E> {
+    public List<E> newElementList(int size);
+    public void addElement(List<E> a, E element);
+    
+    public boolean visitEnter(LikeParseNode node) throws SQLException;
+    public E visitLeave(LikeParseNode node, List<E> l) throws SQLException;
+    
+    public boolean visitEnter(AndParseNode node) throws SQLException;
+    public E visitLeave(AndParseNode node, List<E> l) throws SQLException;
+    
+    public boolean visitEnter(OrParseNode node) throws SQLException;
+    public E visitLeave(OrParseNode node, List<E> l) throws SQLException;
+    
+    public boolean visitEnter(FunctionParseNode node) throws SQLException;
+    public E visitLeave(FunctionParseNode node, List<E> l) throws SQLException;
+    
+    public boolean visitEnter(ComparisonParseNode node) throws SQLException;
+    public E visitLeave(ComparisonParseNode node, List<E> l) throws SQLException;
+
+    public boolean visitEnter(CaseParseNode node) throws SQLException;
+    public E visitLeave(CaseParseNode node, List<E> l) throws SQLException;
+    
+    public boolean visitEnter(CompoundParseNode node) throws SQLException;
+    public E visitLeave(CompoundParseNode node, List<E> l) throws SQLException;
+    
+    public boolean visitEnter(AddParseNode node) throws SQLException;
+    public E visitLeave(AddParseNode node, List<E> l) throws SQLException;
+    
+    public boolean visitEnter(MultiplyParseNode node) throws SQLException;
+    public E visitLeave(MultiplyParseNode node, List<E> l) throws SQLException;
+    
+    public boolean visitEnter(DivideParseNode node) throws SQLException;
+    public E visitLeave(DivideParseNode node, List<E> l) throws SQLException;
+    
+    public boolean visitEnter(SubtractParseNode node) throws SQLException;
+    public E visitLeave(SubtractParseNode node, List<E> l) throws SQLException;
+    
+    public boolean visitEnter(NotParseNode node) throws SQLException;
+    public E visitLeave(NotParseNode node, List<E> l) throws SQLException;
+    
+    public boolean visitEnter(InListParseNode node) throws SQLException;
+    public E visitLeave(InListParseNode node, List<E> l) throws SQLException;
+    
+    public boolean visitEnter(IsNullParseNode node) throws SQLException;
+    public E visitLeave(IsNullParseNode node, List<E> l) throws SQLException;
+    
+    public E visit(ColumnParseNode node) throws SQLException;
+    public E visit(LiteralParseNode node) throws SQLException;
+    public E visit(BindParseNode node) throws SQLException;
+    public E visit(WildcardParseNode node) throws SQLException;  
+    public E visit(FamilyWildcardParseNode node) throws SQLException;  
+    public E visit(ParseNode node) throws SQLException;  
+    
+    public boolean visitEnter(StringConcatParseNode node) throws SQLException;
+    public E visitLeave(StringConcatParseNode node, List<E> l) throws SQLException;
+	
+    public boolean visitEnter(BetweenParseNode node) throws SQLException;
+    public E visitLeave(BetweenParseNode node, List<E> l) throws SQLException;
+    
+    public boolean visitEnter(CastParseNode node) throws SQLException;
+    public E visitLeave(CastParseNode node, List<E> l) throws SQLException;
+    
+    public boolean visitEnter(RowValueConstructorParseNode node) throws SQLException;
+    public E visitLeave(RowValueConstructorParseNode node, List<E> l) throws SQLException;
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/c5b80246/src/main/java/org/apache/phoenix/parse/PrimaryKeyConstraint.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/phoenix/parse/PrimaryKeyConstraint.java b/src/main/java/org/apache/phoenix/parse/PrimaryKeyConstraint.java
new file mode 100644
index 0000000..97c99cb
--- /dev/null
+++ b/src/main/java/org/apache/phoenix/parse/PrimaryKeyConstraint.java
@@ -0,0 +1,69 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.phoenix.parse;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+
+import org.apache.hadoop.hbase.util.Pair;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Maps;
+import org.apache.phoenix.schema.ColumnModifier;
+
+public class PrimaryKeyConstraint extends NamedNode {
+    public static final PrimaryKeyConstraint EMPTY = new PrimaryKeyConstraint(null, Collections.<Pair<ColumnName, ColumnModifier>>emptyList());
+
+    private final List<Pair<ColumnName, ColumnModifier>> columns;
+    private final HashMap<ColumnName, Pair<ColumnName, ColumnModifier>> columnNameToModifier;
+    
+    PrimaryKeyConstraint(String name, List<Pair<ColumnName, ColumnModifier>> columns) {
+        super(name);
+        this.columns = columns == null ? Collections.<Pair<ColumnName, ColumnModifier>>emptyList() : ImmutableList.copyOf(columns);
+        this.columnNameToModifier = Maps.newHashMapWithExpectedSize(this.columns.size());
+        for (Pair<ColumnName, ColumnModifier> p : this.columns) {
+            this.columnNameToModifier.put(p.getFirst(), p);
+        }
+    }
+
+    public List<Pair<ColumnName, ColumnModifier>> getColumnNames() {
+        return columns;
+    }
+    
+    public Pair<ColumnName, ColumnModifier> getColumn(ColumnName columnName) {
+    	return columnNameToModifier.get(columnName);
+    }
+    
+    public boolean contains(ColumnName columnName) {
+        return columnNameToModifier.containsKey(columnName);
+    }
+    
+    @Override
+    public int hashCode() {
+        return super.hashCode();
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+        return super.equals(obj);
+    }
+    
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/c5b80246/src/main/java/org/apache/phoenix/parse/PropertyName.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/phoenix/parse/PropertyName.java b/src/main/java/org/apache/phoenix/parse/PropertyName.java
new file mode 100644
index 0000000..167b323
--- /dev/null
+++ b/src/main/java/org/apache/phoenix/parse/PropertyName.java
@@ -0,0 +1,25 @@
+package org.apache.phoenix.parse;
+
+import org.apache.phoenix.util.SchemaUtil;
+
+public class PropertyName {
+    private final NamedNode familyName;
+    private final String propertyName;
+    
+    PropertyName(String familyName, String propertyName) {
+        this.familyName = familyName == null ? null : new NamedNode(familyName);
+        this.propertyName = SchemaUtil.normalizeIdentifier(propertyName);;
+    }
+
+    PropertyName(String columnName) {
+        this(null, columnName);
+    }
+
+    public String getFamilyName() {
+        return familyName == null ? "" : familyName.getName();
+    }
+
+    public String getPropertyName() {
+        return propertyName;
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/c5b80246/src/main/java/org/apache/phoenix/parse/RowValueConstructorParseNode.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/phoenix/parse/RowValueConstructorParseNode.java b/src/main/java/org/apache/phoenix/parse/RowValueConstructorParseNode.java
new file mode 100644
index 0000000..e35646c
--- /dev/null
+++ b/src/main/java/org/apache/phoenix/parse/RowValueConstructorParseNode.java
@@ -0,0 +1,48 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.phoenix.parse;
+
+import java.sql.SQLException;
+import java.util.Collections;
+import java.util.List;
+
+/**
+ * 
+ * Node representing a row value constructor in SQL.  
+ *
+ * @author samarth.jain
+ * @since 0.1
+ */
+public class RowValueConstructorParseNode extends CompoundParseNode {
+    
+    public RowValueConstructorParseNode(List<ParseNode> l) {
+        super(l);
+    }
+
+    @Override
+    public <T> T accept(ParseNodeVisitor<T> visitor) throws SQLException {
+        List<T> l = Collections.emptyList();
+        if (visitor.visitEnter(this)) {
+            l = acceptChildren(visitor);
+        }
+        return visitor.visitLeave(this, l);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/c5b80246/src/main/java/org/apache/phoenix/parse/SQLParser.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/phoenix/parse/SQLParser.java b/src/main/java/org/apache/phoenix/parse/SQLParser.java
new file mode 100644
index 0000000..dc175a4
--- /dev/null
+++ b/src/main/java/org/apache/phoenix/parse/SQLParser.java
@@ -0,0 +1,173 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.phoenix.parse;
+
+import java.io.IOException;
+import java.io.Reader;
+import java.io.StringReader;
+import java.sql.SQLException;
+import java.sql.SQLFeatureNotSupportedException;
+
+import org.antlr.runtime.ANTLRReaderStream;
+import org.antlr.runtime.CharStream;
+import org.antlr.runtime.CommonTokenStream;
+import org.antlr.runtime.RecognitionException;
+
+import org.apache.phoenix.exception.PhoenixParserException;
+
+/**
+ * 
+ * SQL Parser for Phoenix
+ *
+ * @author jtaylor
+ * @since 0.1
+ */
+public class SQLParser {
+    private static final ParseNodeFactory DEFAULT_NODE_FACTORY = new ParseNodeFactory();
+
+    private final PhoenixSQLParser parser;
+
+    public SQLParser(String query) {
+        this(query,DEFAULT_NODE_FACTORY);
+    }
+
+    public SQLParser(String query, ParseNodeFactory factory) {
+        PhoenixSQLLexer lexer;
+        try {
+            lexer = new PhoenixSQLLexer(new CaseInsensitiveReaderStream(new StringReader(query)));
+        } catch (IOException e) {
+            throw new RuntimeException(e); // Impossible
+        }
+        CommonTokenStream cts = new CommonTokenStream(lexer);
+        parser = new PhoenixSQLParser(cts);
+        parser.setParseNodeFactory(factory);
+    }
+
+    public SQLParser(Reader queryReader, ParseNodeFactory factory) throws IOException {
+        PhoenixSQLLexer lexer = new PhoenixSQLLexer(new CaseInsensitiveReaderStream(queryReader));
+        CommonTokenStream cts = new CommonTokenStream(lexer);
+        parser = new PhoenixSQLParser(cts);
+        parser.setParseNodeFactory(factory);
+    }
+
+    public SQLParser(Reader queryReader) throws IOException {
+        PhoenixSQLLexer lexer = new PhoenixSQLLexer(new CaseInsensitiveReaderStream(queryReader));
+        CommonTokenStream cts = new CommonTokenStream(lexer);
+        parser = new PhoenixSQLParser(cts);
+        parser.setParseNodeFactory(DEFAULT_NODE_FACTORY);
+    }
+
+    /**
+     * Parses the input as a series of semicolon-terminated SQL statements.
+     * @throws SQLException 
+     */
+    public BindableStatement nextStatement(ParseNodeFactory factory) throws SQLException {
+        try {
+            parser.resetBindCount();
+            parser.setParseNodeFactory(factory);
+            BindableStatement statement = parser.nextStatement();
+            return statement;
+        } catch (RecognitionException e) {
+            throw new PhoenixParserException(e, parser);
+        } catch (UnsupportedOperationException e) {
+            throw new SQLFeatureNotSupportedException(e);
+        } catch (RuntimeException e) {
+            if (e.getCause() instanceof SQLException) {
+                throw (SQLException) e.getCause();
+            }
+            throw new PhoenixParserException(e, parser);
+        }
+    }
+
+    /**
+     * Parses the input as a SQL select or upsert statement.
+     * @throws SQLException 
+     */
+    public BindableStatement parseStatement() throws SQLException {
+        try {
+            BindableStatement statement = parser.statement();
+            return statement;
+        } catch (RecognitionException e) {
+            throw new PhoenixParserException(e, parser);
+        } catch (UnsupportedOperationException e) {
+            throw new SQLFeatureNotSupportedException(e);
+        } catch (RuntimeException e) {
+            if (e.getCause() instanceof SQLException) {
+                throw (SQLException) e.getCause();
+            }
+            throw new PhoenixParserException(e, parser);
+        }
+    }
+
+    /**
+     * Parses the input as a SQL select statement.
+     * Used only in tests
+     * @throws SQLException 
+     */
+    public SelectStatement parseQuery() throws SQLException {
+        try {
+            SelectStatement statement = parser.query();
+            return statement;
+        } catch (RecognitionException e) {
+            throw new PhoenixParserException(e, parser);
+        } catch (RuntimeException e) {
+            if (e.getCause() instanceof SQLException) {
+                throw (SQLException) e.getCause();
+            }
+            throw new PhoenixParserException(e, parser);
+        }
+    }
+
+    /**
+     * Parses the input as a SQL literal
+     * @throws SQLException 
+     */
+    public LiteralParseNode parseLiteral() throws SQLException {
+        try {
+            LiteralParseNode literalNode = parser.literal();
+            return literalNode;
+        } catch (RecognitionException e) {
+            throw new PhoenixParserException(e, parser);
+        } catch (RuntimeException e) {
+            if (e.getCause() instanceof SQLException) {
+                throw (SQLException) e.getCause();
+            }
+            throw new PhoenixParserException(e, parser);
+        }
+    }
+
+    private static class CaseInsensitiveReaderStream extends ANTLRReaderStream {
+        CaseInsensitiveReaderStream(Reader script) throws IOException {
+            super(script);
+        }
+
+        @Override
+        public int LA(int i) {
+            if (i == 0) { return 0; // undefined
+            }
+            if (i < 0) {
+                i++; // e.g., translate LA(-1) to use offset 0
+            }
+
+            if ((p + i - 1) >= n) { return CharStream.EOF; }
+            return Character.toLowerCase(data[p + i - 1]);
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/c5b80246/src/main/java/org/apache/phoenix/parse/SelectStatement.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/phoenix/parse/SelectStatement.java b/src/main/java/org/apache/phoenix/parse/SelectStatement.java
new file mode 100644
index 0000000..d11b359
--- /dev/null
+++ b/src/main/java/org/apache/phoenix/parse/SelectStatement.java
@@ -0,0 +1,184 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.phoenix.parse;
+
+import java.util.Collections;
+import java.util.List;
+
+import com.google.common.collect.Lists;
+import org.apache.phoenix.expression.function.CountAggregateFunction;
+import org.apache.phoenix.parse.FunctionParseNode.BuiltInFunction;
+import org.apache.phoenix.parse.FunctionParseNode.BuiltInFunctionInfo;
+
+/**
+ * 
+ * Top level node representing a SQL statement
+ *
+ * @author jtaylor
+ * @since 0.1
+ */
+public class SelectStatement implements FilterableStatement {
+    public static final SelectStatement SELECT_ONE =
+            new SelectStatement(
+                    Collections.<TableNode>emptyList(), null, false, 
+                    Collections.<AliasedNode>singletonList(new AliasedNode(null,new LiteralParseNode(1))),
+                    null, Collections.<ParseNode>emptyList(),
+                    null, Collections.<OrderByNode>emptyList(),
+                    null, 0, false);
+    public static final SelectStatement COUNT_ONE =
+            new SelectStatement(
+                    Collections.<TableNode>emptyList(), null, false,
+                    Collections.<AliasedNode>singletonList(
+                    new AliasedNode(null, 
+                        new AggregateFunctionParseNode(
+                                CountAggregateFunction.NORMALIZED_NAME, 
+                                LiteralParseNode.STAR, 
+                                new BuiltInFunctionInfo(CountAggregateFunction.class, CountAggregateFunction.class.getAnnotation(BuiltInFunction.class))))),
+                    null, Collections.<ParseNode>emptyList(), 
+                    null, Collections.<OrderByNode>emptyList(), 
+                    null, 0, true);
+    public static SelectStatement create(SelectStatement select, HintNode hint) {
+        if (select.getHint() == hint || hint.isEmpty()) {
+            return select;
+        }
+        return new SelectStatement(select.getFrom(), hint, select.isDistinct(), 
+                select.getSelect(), select.getWhere(), select.getGroupBy(), select.getHaving(), 
+                select.getOrderBy(), select.getLimit(), select.getBindCount(), select.isAggregate());
+    }
+    
+    private final List<TableNode> fromTable;
+    private final HintNode hint;
+    private final boolean isDistinct;
+    private final List<AliasedNode> select;
+    private final ParseNode where;
+    private final List<ParseNode> groupBy;
+    private final ParseNode having;
+    private final List<OrderByNode> orderBy;
+    private final LimitNode limit;
+    private final int bindCount;
+    private final boolean isAggregate;
+    
+    // Filter out constants from GROUP BY as they're useless
+    private static List<ParseNode> filterGroupByConstants(List<ParseNode> nodes) {
+        List<ParseNode> filteredNodes = nodes;
+        for (int i = 0; i < nodes.size(); i++) {
+            ParseNode node = nodes.get(i);
+            if (node.isConstant()) {
+                if (filteredNodes == nodes) {
+                    filteredNodes = Lists.newArrayListWithExpectedSize(nodes.size());
+                    filteredNodes.addAll(nodes.subList(0, i));
+                }
+            } else if (filteredNodes != nodes) {
+                filteredNodes.add(node);
+            }
+        }
+        return filteredNodes;
+    }
+    
+    // Filter out constants from ORDER BY as they're useless
+    private static List<OrderByNode> filterOrderByConstants(List<OrderByNode> nodes) {
+        List<OrderByNode> filteredNodes = nodes;
+        for (int i = 0; i < nodes.size(); i++) {
+            ParseNode node = nodes.get(i).getNode();
+            if (node.isConstant()) {
+                if (filteredNodes == nodes) {
+                    filteredNodes = Lists.newArrayListWithExpectedSize(nodes.size());
+                    filteredNodes.addAll(nodes.subList(0, i));
+                }
+            } else if (filteredNodes != nodes) {
+                filteredNodes.add(nodes.get(i));
+            }
+        }
+        return filteredNodes;
+    }
+    
+    protected SelectStatement(List<? extends TableNode> from, HintNode hint, boolean isDistinct, List<AliasedNode> select, ParseNode where, List<ParseNode> groupBy, ParseNode having, List<OrderByNode> orderBy, LimitNode limit, int bindCount, boolean isAggregate) {
+        this.fromTable = Collections.unmodifiableList(from);
+        this.hint = hint == null ? HintNode.EMPTY_HINT_NODE : hint;
+        this.isDistinct = isDistinct;
+        this.select = Collections.unmodifiableList(select);
+        this.where = where;
+        this.groupBy = Collections.unmodifiableList(filterGroupByConstants(groupBy));
+        this.having = having;
+        this.orderBy = Collections.unmodifiableList(filterOrderByConstants(orderBy));
+        this.limit = limit;
+        this.bindCount = bindCount;
+        this.isAggregate = isAggregate || !this.groupBy.isEmpty() || this.having != null;
+    }
+    
+    @Override
+    public boolean isDistinct() {
+        return isDistinct;
+    }
+    
+    @Override
+    public LimitNode getLimit() {
+        return limit;
+    }
+    
+    @Override
+    public int getBindCount() {
+        return bindCount;
+    }
+    
+    public List<TableNode> getFrom() {
+        return fromTable;
+    }
+    
+    @Override
+    public HintNode getHint() {
+        return hint;
+    }
+    
+    public List<AliasedNode> getSelect() {
+        return select;
+    }
+    /**
+     * Gets the where condition, or null if none.
+     */
+    @Override
+    public ParseNode getWhere() {
+        return where;
+    }
+    
+    /**
+     * Gets the group-by, containing at least 1 element, or null, if none.
+     */
+    public List<ParseNode> getGroupBy() {
+        return groupBy;
+    }
+    
+    public ParseNode getHaving() {
+        return having;
+    }
+    
+    /**
+     * Gets the order-by, containing at least 1 element, or null, if none.
+     */
+    @Override
+    public List<OrderByNode> getOrderBy() {
+        return orderBy;
+    }
+
+    @Override
+    public boolean isAggregate() {
+        return isAggregate;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/c5b80246/src/main/java/org/apache/phoenix/parse/SelectStatementRewriter.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/phoenix/parse/SelectStatementRewriter.java b/src/main/java/org/apache/phoenix/parse/SelectStatementRewriter.java
new file mode 100644
index 0000000..1d18154
--- /dev/null
+++ b/src/main/java/org/apache/phoenix/parse/SelectStatementRewriter.java
@@ -0,0 +1,188 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.phoenix.parse;
+
+import java.sql.SQLException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Set;
+
+
+
+/**
+ * 
+ * Class that creates a new select statement by filtering out nodes.
+ * Currently only supports filtering out boolean nodes (i.e. nodes
+ * that may be ANDed and ORed together.
+ *
+ * TODO: generize this
+ * @author jtaylor
+ * @since 0.1
+ */
+public class SelectStatementRewriter extends ParseNodeRewriter {
+    
+    /**
+     * Rewrite the select statement by filtering out expression nodes from the WHERE clause
+     * @param statement the select statement from which to filter.
+     * @param removeNodes expression nodes to filter out of WHERE clause.
+     * @return new select statement
+     * @throws SQLException 
+     */
+    public static SelectStatement removeFromWhereClause(SelectStatement statement, Set<ParseNode> removeNodes) throws SQLException {
+        if (removeNodes.isEmpty()) {
+            return statement;
+        }
+        ParseNode where = statement.getWhere();
+        SelectStatementRewriter rewriter = new SelectStatementRewriter(removeNodes);
+        where = where.accept(rewriter);
+        // Return new SELECT statement with updated WHERE clause
+        return NODE_FACTORY.select(statement, where, statement.getHaving());
+    }
+    
+    /**
+     * Rewrite the select statement by filtering out expression nodes from the HAVING clause
+     * and anding them with the WHERE clause.
+     * @param statement the select statement from which to move the nodes.
+     * @param moveNodes expression nodes to filter out of HAVING clause and add to WHERE clause.
+     * @return new select statement
+     * @throws SQLException 
+     */
+    public static SelectStatement moveFromHavingToWhereClause(SelectStatement statement, Set<ParseNode> moveNodes) throws SQLException {
+        if (moveNodes.isEmpty()) {
+            return statement;
+        }
+        ParseNode andNode = NODE_FACTORY.and(new ArrayList<ParseNode>(moveNodes));
+        ParseNode having = statement.getHaving();
+        SelectStatementRewriter rewriter = new SelectStatementRewriter(moveNodes);
+        having = having.accept(rewriter);
+        ParseNode where = statement.getWhere();
+        if (where == null) {
+            where = andNode;
+        } else {
+            where = NODE_FACTORY.and(Arrays.asList(where,andNode));
+        }
+        // Return new SELECT statement with updated WHERE and HAVING clauses
+        return NODE_FACTORY.select(statement, where, having);
+    }
+    
+    private static final ParseNodeFactory NODE_FACTORY = new ParseNodeFactory();
+
+    private final Set<ParseNode> removeNodes;
+    
+    private SelectStatementRewriter(Set<ParseNode> removeNodes) {
+        this.removeNodes = removeNodes;
+    }
+    
+    private static interface CompoundNodeFactory {
+        ParseNode createNode(List<ParseNode> children);
+    }
+    
+    private boolean enterCompoundNode(ParseNode node) {
+        if (removeNodes.contains(node)) {
+            return false;
+        }
+        return true;
+    }
+    
+    private ParseNode leaveCompoundNode(CompoundParseNode node, List<ParseNode> children, CompoundNodeFactory factory) {
+        int newSize = children.size();
+        int oldSize = node.getChildren().size();
+        if (newSize == oldSize) {
+            return node;
+        } else if (newSize > 1) {
+            return factory.createNode(children);
+        } else if (newSize == 1) {
+            // TODO: keep or collapse? Maybe be helpful as context of where a problem occurs if a node could not be consumed
+            return(children.get(0));
+        } else {
+            return null;
+        }
+    }
+    
+    @Override
+    public boolean visitEnter(AndParseNode node) throws SQLException {
+        return enterCompoundNode(node);
+    }
+
+    @Override
+    public ParseNode visitLeave(AndParseNode node, List<ParseNode> nodes) throws SQLException {
+        return leaveCompoundNode(node, nodes, new CompoundNodeFactory() {
+            @Override
+            public ParseNode createNode(List<ParseNode> children) {
+                return NODE_FACTORY.and(children);
+            }
+        });
+    }
+
+    @Override
+    public boolean visitEnter(OrParseNode node) throws SQLException {
+        return enterCompoundNode(node);
+    }
+
+    @Override
+    public ParseNode visitLeave(OrParseNode node, List<ParseNode> nodes) throws SQLException {
+        return leaveCompoundNode(node, nodes, new CompoundNodeFactory() {
+            @Override
+            public ParseNode createNode(List<ParseNode> children) {
+                return NODE_FACTORY.or(children);
+            }
+        });
+    }
+    
+    @Override
+    public boolean visitEnter(ComparisonParseNode node) throws SQLException {
+        if (removeNodes.contains(node)) {
+            return false;
+        }
+        return true;
+    }
+
+    @Override
+    public ParseNode visitLeave(ComparisonParseNode node, List<ParseNode> c) throws SQLException {
+        return c.isEmpty() ? null : node;
+    }
+    
+    @Override
+    public boolean visitEnter(LikeParseNode node) throws SQLException {
+        if (removeNodes.contains(node)) {
+            return false;
+        }
+        return true;
+    }
+    
+    @Override
+    public ParseNode visitLeave(LikeParseNode node, List<ParseNode> c) throws SQLException {
+        return c.isEmpty() ? null : node;
+    }
+    
+    @Override
+    public boolean visitEnter(InListParseNode node) throws SQLException {
+        if (removeNodes.contains(node)) {
+            return false;
+        }
+        return true;
+    }
+    
+    @Override
+    public ParseNode visitLeave(InListParseNode node, List<ParseNode> c) throws SQLException {
+        return c.isEmpty() ? null : node;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/c5b80246/src/main/java/org/apache/phoenix/parse/ShowTablesStatement.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/phoenix/parse/ShowTablesStatement.java b/src/main/java/org/apache/phoenix/parse/ShowTablesStatement.java
new file mode 100644
index 0000000..668721e
--- /dev/null
+++ b/src/main/java/org/apache/phoenix/parse/ShowTablesStatement.java
@@ -0,0 +1,8 @@
+package org.apache.phoenix.parse;
+
+public class ShowTablesStatement implements BindableStatement {
+    @Override
+    public int getBindCount() {
+        return 0;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/c5b80246/src/main/java/org/apache/phoenix/parse/SingleTableSQLStatement.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/phoenix/parse/SingleTableSQLStatement.java b/src/main/java/org/apache/phoenix/parse/SingleTableSQLStatement.java
new file mode 100644
index 0000000..ec5976e
--- /dev/null
+++ b/src/main/java/org/apache/phoenix/parse/SingleTableSQLStatement.java
@@ -0,0 +1,39 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.phoenix.parse;
+
+public abstract class SingleTableSQLStatement implements BindableStatement {
+    private final NamedTableNode table;
+    private final int bindCount;
+
+    public SingleTableSQLStatement(NamedTableNode table, int bindCount) {
+        this.table = table;
+        this.bindCount = bindCount;
+    }
+    
+    public NamedTableNode getTable() {
+        return table;
+    }
+
+    @Override
+    public int getBindCount() {
+        return bindCount;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/c5b80246/src/main/java/org/apache/phoenix/parse/StatelessTraverseAllParseNodeVisitor.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/phoenix/parse/StatelessTraverseAllParseNodeVisitor.java b/src/main/java/org/apache/phoenix/parse/StatelessTraverseAllParseNodeVisitor.java
new file mode 100644
index 0000000..a9eb8ec
--- /dev/null
+++ b/src/main/java/org/apache/phoenix/parse/StatelessTraverseAllParseNodeVisitor.java
@@ -0,0 +1,86 @@
+package org.apache.phoenix.parse;
+
+import java.sql.SQLException;
+import java.util.List;
+
+public class StatelessTraverseAllParseNodeVisitor extends TraverseAllParseNodeVisitor<Void> {
+    @Override
+    public Void visitLeave(LikeParseNode node, List<Void> l) throws SQLException {
+        return null;
+    }
+
+    @Override
+    public Void visitLeave(AndParseNode node, List<Void> l) throws SQLException {
+        return null;
+    }
+
+    @Override
+    public Void visitLeave(OrParseNode node, List<Void> l) throws SQLException {
+        return null;
+    }
+
+    @Override
+    public Void visitLeave(FunctionParseNode node, List<Void> l) throws SQLException {
+        return null;
+    }
+
+    @Override
+    public Void visitLeave(ComparisonParseNode node, List<Void> l) throws SQLException {
+        return null;
+    }
+
+    @Override
+    public Void visitLeave(CaseParseNode node, List<Void> l) throws SQLException {
+        return null;
+    }
+
+    @Override
+    public Void visitLeave(AddParseNode node, List<Void> l) throws SQLException {
+        return null;
+    }
+
+    @Override
+    public Void visitLeave(MultiplyParseNode node, List<Void> l) throws SQLException {
+        return null;
+    }
+
+    @Override
+    public Void visitLeave(DivideParseNode node, List<Void> l) throws SQLException {
+        return null;
+    }
+
+    @Override
+    public Void visitLeave(SubtractParseNode node, List<Void> l) throws SQLException {
+        return null;
+    }
+
+    @Override
+    public Void visitLeave(NotParseNode node, List<Void> l) throws SQLException {
+        return null;
+    }
+    
+    @Override
+    public Void visitLeave(CastParseNode node, List<Void> l) throws SQLException {
+        return null;
+    }
+    
+    @Override
+    public Void visitLeave(InListParseNode node, List<Void> l) throws SQLException {
+        return null;
+    }
+
+    @Override
+    public Void visitLeave(StringConcatParseNode node, List<Void> l) throws SQLException {
+        return null;
+    }
+
+    @Override
+    public Void visitLeave(BetweenParseNode node, List<Void> l) throws SQLException {
+        return null;
+    }
+
+    @Override
+    public Void visitLeave(RowValueConstructorParseNode node, List<Void> l) throws SQLException {
+        return null;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/c5b80246/src/main/java/org/apache/phoenix/parse/StringConcatParseNode.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/phoenix/parse/StringConcatParseNode.java b/src/main/java/org/apache/phoenix/parse/StringConcatParseNode.java
new file mode 100644
index 0000000..b5134d2
--- /dev/null
+++ b/src/main/java/org/apache/phoenix/parse/StringConcatParseNode.java
@@ -0,0 +1,51 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.phoenix.parse;
+
+import java.sql.SQLException;
+import java.util.Collections;
+import java.util.List;
+
+
+
+
+/**
+ * 
+ * Node representing || String concatenation in a SQL expression
+ *
+ * @author kmahadik
+ * @since 0.1
+ */
+public class StringConcatParseNode extends CompoundParseNode {
+
+    StringConcatParseNode(List<ParseNode> children) {
+        super(children);
+    }
+
+    @Override
+    public <T> T accept(ParseNodeVisitor<T> visitor) throws SQLException {
+        List<T> l = Collections.emptyList();
+        if (visitor.visitEnter(this)) {
+            l = acceptChildren(visitor);
+        }
+        return visitor.visitLeave(this, l);
+    }
+    
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/c5b80246/src/main/java/org/apache/phoenix/parse/SubqueryParseNode.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/phoenix/parse/SubqueryParseNode.java b/src/main/java/org/apache/phoenix/parse/SubqueryParseNode.java
new file mode 100644
index 0000000..b2073df
--- /dev/null
+++ b/src/main/java/org/apache/phoenix/parse/SubqueryParseNode.java
@@ -0,0 +1,49 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.phoenix.parse;
+
+import java.sql.SQLException;
+
+
+
+/**
+ * 
+ * Node representing a subquery in SQL
+ *
+ * @author jtaylor
+ * @since 0.1
+ */
+public class SubqueryParseNode extends TerminalParseNode {
+    private final SelectStatement select;
+
+    SubqueryParseNode(SelectStatement select) {
+        this.select = select;
+    }
+    
+    public SelectStatement getSelectNode() {
+        return select;
+    }
+
+    @Override
+    public <T> T accept(ParseNodeVisitor<T> visitor) throws SQLException {
+        return visitor.visit(this);
+    }
+    
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/c5b80246/src/main/java/org/apache/phoenix/parse/SubtractParseNode.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/phoenix/parse/SubtractParseNode.java b/src/main/java/org/apache/phoenix/parse/SubtractParseNode.java
new file mode 100644
index 0000000..c12efef
--- /dev/null
+++ b/src/main/java/org/apache/phoenix/parse/SubtractParseNode.java
@@ -0,0 +1,48 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.phoenix.parse;
+
+import java.sql.SQLException;
+import java.util.Collections;
+import java.util.List;
+
+
+
+/**
+ * 
+ * Node representing subtraction in a SQL expression
+ *
+ * @author jtaylor
+ * @since 0.1
+ */
+public class SubtractParseNode extends ArithmeticParseNode {
+    SubtractParseNode(List<ParseNode> children) {
+        super(children);
+    }
+
+    @Override
+    public <T> T accept(ParseNodeVisitor<T> visitor) throws SQLException {
+        List<T> l = Collections.emptyList();
+        if (visitor.visitEnter(this)) {
+            l = acceptChildren(visitor);
+        }
+        return visitor.visitLeave(this, l);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/c5b80246/src/main/java/org/apache/phoenix/parse/SumAggregateParseNode.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/phoenix/parse/SumAggregateParseNode.java b/src/main/java/org/apache/phoenix/parse/SumAggregateParseNode.java
new file mode 100644
index 0000000..8669b8d
--- /dev/null
+++ b/src/main/java/org/apache/phoenix/parse/SumAggregateParseNode.java
@@ -0,0 +1,41 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.phoenix.parse;
+
+import java.sql.SQLException;
+import java.util.List;
+
+import org.apache.phoenix.compile.StatementContext;
+import org.apache.phoenix.expression.Expression;
+import org.apache.phoenix.expression.function.FunctionExpression;
+import org.apache.phoenix.expression.function.SumAggregateFunction;
+
+
+public class SumAggregateParseNode extends DelegateConstantToCountParseNode {
+
+    public SumAggregateParseNode(String name, List<ParseNode> children, BuiltInFunctionInfo info) {
+        super(name, children, info);
+    }
+    
+    @Override
+    public FunctionExpression create(List<Expression> children, StatementContext context) throws SQLException {
+        return new SumAggregateFunction(children, getDelegateFunction(children,context));
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/c5b80246/src/main/java/org/apache/phoenix/parse/TableName.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/phoenix/parse/TableName.java b/src/main/java/org/apache/phoenix/parse/TableName.java
new file mode 100644
index 0000000..edb357c
--- /dev/null
+++ b/src/main/java/org/apache/phoenix/parse/TableName.java
@@ -0,0 +1,56 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.phoenix.parse;
+
+import org.apache.phoenix.query.QueryConstants;
+import org.apache.phoenix.util.SchemaUtil;
+
+public class TableName {
+    private final String tableName;
+    private final String schemaName;
+    
+    public static TableName createNormalized(String schemaName, String tableName) {
+        schemaName = schemaName == null ? null : SchemaUtil.normalizeIdentifier(schemaName);
+        tableName = SchemaUtil.normalizeIdentifier(tableName);
+        return new TableName(schemaName, tableName);
+    }
+    
+    public static TableName create(String schemaName, String tableName) {
+        return new TableName(schemaName,tableName);
+    }
+    
+    private TableName(String schemaName, String tableName) {
+        this.schemaName = schemaName;
+        this.tableName = tableName;
+    }
+
+    public String getTableName() {
+        return tableName;
+    }
+
+    public String getSchemaName() {
+        return schemaName;
+    }
+    
+    @Override
+    public String toString() {
+        return (schemaName == null ? "" : schemaName + QueryConstants.NAME_SEPARATOR)  + tableName;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/c5b80246/src/main/java/org/apache/phoenix/parse/TableNode.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/phoenix/parse/TableNode.java b/src/main/java/org/apache/phoenix/parse/TableNode.java
new file mode 100644
index 0000000..0eaf140
--- /dev/null
+++ b/src/main/java/org/apache/phoenix/parse/TableNode.java
@@ -0,0 +1,45 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.phoenix.parse;
+
+import java.sql.SQLException;
+
+
+
+/**
+ * 
+ * Abstract base class for FROM clause data sources
+ *
+ * @author jtaylor
+ * @since 0.1
+ */
+public abstract class TableNode {
+    private final String alias;
+
+    TableNode(String alias) {
+        this.alias = alias;
+    }
+
+    public String getAlias() {
+        return alias;
+    }
+
+    public abstract void accept(TableNodeVisitor visitor) throws SQLException;
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/c5b80246/src/main/java/org/apache/phoenix/parse/TableNodeVisitor.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/phoenix/parse/TableNodeVisitor.java b/src/main/java/org/apache/phoenix/parse/TableNodeVisitor.java
new file mode 100644
index 0000000..6a14da8
--- /dev/null
+++ b/src/main/java/org/apache/phoenix/parse/TableNodeVisitor.java
@@ -0,0 +1,37 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.phoenix.parse;
+
+import java.sql.SQLException;
+
+
+/**
+ * 
+ * Visitor for nodes in the FROM clause
+ *
+ * @author jtaylor
+ * @since 0.1
+ */
+public interface TableNodeVisitor {
+    void visit(BindTableNode boundTableNode) throws SQLException;
+    void visit(JoinTableNode joinNode) throws SQLException;
+    void visit(NamedTableNode namedTableNode) throws SQLException;
+    void visit(DerivedTableNode subselectNode) throws SQLException;
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/c5b80246/src/main/java/org/apache/phoenix/parse/TerminalParseNode.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/phoenix/parse/TerminalParseNode.java b/src/main/java/org/apache/phoenix/parse/TerminalParseNode.java
new file mode 100644
index 0000000..2c31341
--- /dev/null
+++ b/src/main/java/org/apache/phoenix/parse/TerminalParseNode.java
@@ -0,0 +1,37 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.phoenix.parse;
+
+import java.util.Collections;
+import java.util.List;
+
+/**
+ * 
+ * Abstract node for expressions that have no children
+ *
+ * @author jtaylor
+ * @since 0.1
+ */
+public abstract class TerminalParseNode extends ParseNode {
+    @Override
+    public final List<ParseNode> getChildren() {
+        return Collections.emptyList();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/c5b80246/src/main/java/org/apache/phoenix/parse/ToCharParseNode.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/phoenix/parse/ToCharParseNode.java b/src/main/java/org/apache/phoenix/parse/ToCharParseNode.java
new file mode 100644
index 0000000..ec2c950
--- /dev/null
+++ b/src/main/java/org/apache/phoenix/parse/ToCharParseNode.java
@@ -0,0 +1,65 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.phoenix.parse;
+
+import java.sql.SQLException;
+import java.text.Format;
+import java.util.List;
+
+import org.apache.phoenix.compile.StatementContext;
+import org.apache.phoenix.expression.Expression;
+import org.apache.phoenix.expression.LiteralExpression;
+import org.apache.phoenix.expression.function.*;
+import org.apache.phoenix.schema.PDataType;
+
+
+public class ToCharParseNode extends FunctionParseNode {
+
+    public ToCharParseNode(String name, List<ParseNode> children, BuiltInFunctionInfo info) {
+        super(name, children, info);
+    }
+
+    @Override
+    public FunctionExpression create(List<Expression> children, StatementContext context) throws SQLException {
+        PDataType dataType = children.get(0).getDataType();
+        String formatString = (String)((LiteralExpression)children.get(1)).getValue(); // either date or number format string
+        Format formatter;
+        FunctionArgumentType type;
+        if (dataType.isCoercibleTo(PDataType.TIMESTAMP)) {
+            if (formatString == null) {
+                formatString = context.getDateFormat();
+                formatter = context.getDateFormatter();
+            } else {
+                formatter = FunctionArgumentType.TEMPORAL.getFormatter(formatString);
+            }
+            type = FunctionArgumentType.TEMPORAL;
+        }
+        else if (dataType.isCoercibleTo(PDataType.DECIMAL)) {
+            if (formatString == null)
+                formatString = context.getNumberFormat();
+            formatter = FunctionArgumentType.NUMERIC.getFormatter(formatString);
+            type = FunctionArgumentType.NUMERIC;
+        }
+        else {
+            throw new SQLException(dataType + " type is unsupported for TO_CHAR().  Numeric and temporal types are supported.");
+        }
+        return new ToCharFunction(children, type, formatString, formatter);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/c5b80246/src/main/java/org/apache/phoenix/parse/ToDateParseNode.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/phoenix/parse/ToDateParseNode.java b/src/main/java/org/apache/phoenix/parse/ToDateParseNode.java
new file mode 100644
index 0000000..9863902
--- /dev/null
+++ b/src/main/java/org/apache/phoenix/parse/ToDateParseNode.java
@@ -0,0 +1,51 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.phoenix.parse;
+
+import java.sql.SQLException;
+import java.text.Format;
+import java.util.List;
+
+import org.apache.phoenix.compile.StatementContext;
+import org.apache.phoenix.expression.Expression;
+import org.apache.phoenix.expression.LiteralExpression;
+import org.apache.phoenix.expression.function.FunctionExpression;
+import org.apache.phoenix.expression.function.ToDateFunction;
+import org.apache.phoenix.util.DateUtil;
+
+
+public class ToDateParseNode extends FunctionParseNode {
+    public ToDateParseNode(String name, List<ParseNode> children, BuiltInFunctionInfo info) {
+        super(name, children, info);
+    }
+
+    @Override
+    public FunctionExpression create(List<Expression> children, StatementContext context) throws SQLException {
+        Format dateParser;
+        String dateFormat = (String)((LiteralExpression)children.get(1)).getValue();
+        if (dateFormat == null) {
+            dateFormat = context.getDateFormat();
+            dateParser = context.getDateParser();
+        } else {
+            dateParser = DateUtil.getDateParser(dateFormat);
+        }
+        return new ToDateFunction(children, dateFormat, dateParser);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/c5b80246/src/main/java/org/apache/phoenix/parse/ToNumberParseNode.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/phoenix/parse/ToNumberParseNode.java b/src/main/java/org/apache/phoenix/parse/ToNumberParseNode.java
new file mode 100644
index 0000000..1a079bc
--- /dev/null
+++ b/src/main/java/org/apache/phoenix/parse/ToNumberParseNode.java
@@ -0,0 +1,47 @@
+package org.apache.phoenix.parse;
+
+import java.sql.SQLException;
+import java.text.Format;
+import java.util.List;
+
+import org.apache.phoenix.compile.StatementContext;
+import org.apache.phoenix.expression.Expression;
+import org.apache.phoenix.expression.LiteralExpression;
+import org.apache.phoenix.expression.function.*;
+import org.apache.phoenix.schema.PDataType;
+
+public class ToNumberParseNode extends FunctionParseNode {
+
+    ToNumberParseNode(String name, List<ParseNode> children,
+            BuiltInFunctionInfo info) {
+        super(name, children, info);
+    }
+
+    @Override
+    public FunctionExpression create(List<Expression> children, StatementContext context) throws SQLException {
+        PDataType dataType = children.get(0).getDataType();
+        String formatString = (String)((LiteralExpression)children.get(1)).getValue(); // either date or number format string
+        Format formatter =  null;
+        FunctionArgumentType type;
+        
+        if (dataType.isCoercibleTo(PDataType.TIMESTAMP)) {
+            if (formatString == null) {
+                formatString = context.getDateFormat();
+                formatter = context.getDateFormatter();
+            } else {
+                formatter = FunctionArgumentType.TEMPORAL.getFormatter(formatString);
+            }
+            type = FunctionArgumentType.TEMPORAL;
+        }
+        else if (dataType.isCoercibleTo(PDataType.CHAR)) {
+            if (formatString != null) {
+                formatter = FunctionArgumentType.CHAR.getFormatter(formatString);
+            }
+            type = FunctionArgumentType.CHAR;
+        }
+        else {
+            throw new SQLException(dataType + " type is unsupported for TO_NUMBER().  Numeric and temporal types are supported.");
+        }
+        return new ToNumberFunction(children, type, formatString, formatter);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/c5b80246/src/main/java/org/apache/phoenix/parse/TraverseAllParseNodeVisitor.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/phoenix/parse/TraverseAllParseNodeVisitor.java b/src/main/java/org/apache/phoenix/parse/TraverseAllParseNodeVisitor.java
new file mode 100644
index 0000000..dc5a12f
--- /dev/null
+++ b/src/main/java/org/apache/phoenix/parse/TraverseAllParseNodeVisitor.java
@@ -0,0 +1,148 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.phoenix.parse;
+
+import java.sql.SQLException;
+import java.util.List;
+
+
+/**
+ * 
+ * Visitor that traverses into all parse nodes
+ *
+ * @author jtaylor
+ * @since 0.1
+ */
+public abstract class TraverseAllParseNodeVisitor<T> extends BaseParseNodeVisitor<T> {
+    @Override
+    public boolean visitEnter(AndParseNode node) throws SQLException {
+        return true;
+    }
+
+    @Override
+    public boolean visitEnter(OrParseNode node) throws SQLException {
+        return true;
+    }
+
+    @Override
+    public boolean visitEnter(FunctionParseNode node) throws SQLException {
+        return true;
+    }
+
+    @Override
+    public boolean visitEnter(CaseParseNode node) throws SQLException {
+        return true;
+    }
+
+    @Override
+    public boolean visitEnter(ComparisonParseNode node) throws SQLException {
+        return true;
+    }
+    
+    @Override
+    public boolean visitEnter(LikeParseNode node) throws SQLException {
+        return true;
+    }
+    
+    @Override
+    public boolean visitEnter(NotParseNode node) throws SQLException {
+        return true;
+    }
+    
+    @Override
+    public boolean visitEnter(CastParseNode node) throws SQLException {
+        return true;
+    }
+    
+    @Override
+    public boolean visitEnter(InListParseNode node) throws SQLException {
+        return true;
+    }
+    
+    @Override
+    public boolean visitEnter(IsNullParseNode node) throws SQLException {
+        return true;
+    }
+    
+    @Override
+    public boolean visitEnter(MultiplyParseNode node) throws SQLException {
+        return true;
+    }
+    
+    @Override
+    public boolean visitEnter(SubtractParseNode node) throws SQLException {
+        return true;
+    }
+    
+    @Override
+    public boolean visitEnter(AddParseNode node) throws SQLException {
+        return true;
+    }
+    
+    @Override
+    public boolean visitEnter(DivideParseNode node) throws SQLException {
+        return true;
+    }
+
+    @Override
+    public boolean visitEnter(BetweenParseNode node) throws SQLException {
+        return true;
+    }
+    
+    @Override
+    public T visit(ColumnParseNode node) throws SQLException {
+        return null;
+    }
+
+    @Override
+    public T visitLeave(IsNullParseNode node, List<T> l) throws SQLException {
+        return null;
+    }
+    
+    @Override
+    public T visit(LiteralParseNode node) throws SQLException {
+        return null;
+    }
+
+    @Override
+    public T visit(BindParseNode node) throws SQLException {
+        return null;
+    }
+
+    @Override
+    public T visit(WildcardParseNode node) throws SQLException {
+        return null;
+    }
+
+    @Override
+    public T visit(FamilyWildcardParseNode node) throws SQLException {
+        return null;
+    }
+    
+    @Override
+    public boolean visitEnter(StringConcatParseNode node) throws SQLException {
+        return true;
+    }
+    
+    @Override
+    public boolean visitEnter(RowValueConstructorParseNode node) throws SQLException {
+        return true;
+    }
+}