You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@phoenix.apache.org by ma...@apache.org on 2015/07/04 20:33:44 UTC

phoenix git commit: PHOENIX-922 Support SELECT without a from clause

Repository: phoenix
Updated Branches:
  refs/heads/4.x-HBase-0.98 9ad9445be -> f3056ba67


PHOENIX-922 Support SELECT without a from clause


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

Branch: refs/heads/4.x-HBase-0.98
Commit: f3056ba6728332fbca8521a2c196eb89b8c67f13
Parents: 9ad9445
Author: maryannxue <we...@intel.com>
Authored: Sat Jul 4 14:33:18 2015 -0400
Committer: maryannxue <we...@intel.com>
Committed: Sat Jul 4 14:33:18 2015 -0400

----------------------------------------------------------------------
 .../org/apache/phoenix/end2end/SequenceIT.java  | 27 ++++++
 phoenix-core/src/main/antlr3/PhoenixSQL.g       |  2 +-
 .../apache/phoenix/compile/FromCompiler.java    | 13 +--
 .../phoenix/compile/ProjectionCompiler.java     |  2 +-
 .../apache/phoenix/compile/QueryCompiler.java   |  9 +-
 .../compile/TupleProjectionCompiler.java        |  1 +
 .../apache/phoenix/execute/BaseQueryPlan.java   |  9 +-
 .../phoenix/execute/EmptyTableQueryPlan.java    | 96 ++++++++++++++++++++
 .../apache/phoenix/parse/SelectStatement.java   |  6 +-
 .../org/apache/phoenix/schema/PTableImpl.java   |  1 +
 .../org/apache/phoenix/schema/TableRef.java     |  2 +
 .../phoenix/compile/QueryCompilerTest.java      | 45 +++++++++
 .../apache/phoenix/parse/QueryParserTest.java   | 19 +---
 13 files changed, 195 insertions(+), 37 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/f3056ba6/phoenix-core/src/it/java/org/apache/phoenix/end2end/SequenceIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/SequenceIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/SequenceIT.java
index 6f2ec82..4273022 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/SequenceIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/SequenceIT.java
@@ -1242,5 +1242,32 @@ public class SequenceIT extends BaseClientManagedTimeIT {
         assertEquals(4, rs.getLong(1));
         assertFalse(rs.next());
     }
+    
+    @Test
+    public void testNoFromClause() throws Exception {
+        ResultSet rs;
+        nextConnection();
+        conn.createStatement().execute("CREATE SEQUENCE myseq START WITH 1 INCREMENT BY 1");
+        conn.createStatement().execute("CREATE SEQUENCE anotherseq START WITH 2 INCREMENT BY 3");
+        nextConnection();
+        rs = conn.createStatement().executeQuery("EXPLAIN SELECT NEXT VALUE FOR myseq");
+        assertEquals("CLIENT RESERVE VALUES FROM 1 SEQUENCE", QueryUtil.getExplainPlan(rs));
+        rs = conn.createStatement().executeQuery("SELECT NEXT VALUE FOR myseq");
+        assertTrue(rs.next());
+        assertEquals(1, rs.getInt(1));
+        rs = conn.createStatement().executeQuery("EXPLAIN SELECT CURRENT VALUE FOR myseq");
+        assertEquals("CLIENT RESERVE VALUES FROM 1 SEQUENCE", QueryUtil.getExplainPlan(rs));
+        rs = conn.createStatement().executeQuery("SELECT CURRENT VALUE FOR myseq");
+        assertTrue(rs.next());
+        assertEquals(1, rs.getInt(1));
+        rs = conn.createStatement().executeQuery("SELECT NEXT VALUE FOR myseq, NEXT VALUE FOR anotherseq");
+        assertTrue(rs.next());
+        assertEquals(2, rs.getInt(1));
+        assertEquals(2, rs.getInt(2));
+        rs = conn.createStatement().executeQuery("SELECT CURRENT VALUE FOR myseq, NEXT VALUE FOR anotherseq");
+        assertTrue(rs.next());
+        assertEquals(2, rs.getInt(1));
+        assertEquals(5, rs.getInt(2));        
+    }
 
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/phoenix/blob/f3056ba6/phoenix-core/src/main/antlr3/PhoenixSQL.g
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/antlr3/PhoenixSQL.g b/phoenix-core/src/main/antlr3/PhoenixSQL.g
index 836590f..b6a9c1a 100644
--- a/phoenix-core/src/main/antlr3/PhoenixSQL.g
+++ b/phoenix-core/src/main/antlr3/PhoenixSQL.g
@@ -621,7 +621,7 @@ single_select returns [SelectStatement ret]
 @init{ contextStack.push(new ParseContext()); }
     :   SELECT (h=hintClause)? 
         (d=DISTINCT | ALL)? sel=select_list
-        FROM from=parseFrom
+        (FROM from=parseFrom)?
         (WHERE where=expression)?
         (GROUP BY group=group_by)?
         (HAVING having=expression)?

http://git-wip-us.apache.org/repos/asf/phoenix/blob/f3056ba6/phoenix-core/src/main/java/org/apache/phoenix/compile/FromCompiler.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/FromCompiler.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/FromCompiler.java
index bc753c9..30a2bb7 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/compile/FromCompiler.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/FromCompiler.java
@@ -22,12 +22,9 @@ import java.sql.SQLFeatureNotSupportedException;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.HashMap;
-import java.util.HashSet;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
-import java.util.Set;
-
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.client.HTableInterface;
 import org.apache.phoenix.coprocessor.MetaDataProtocol;
@@ -37,13 +34,11 @@ import org.apache.phoenix.jdbc.PhoenixConnection;
 import org.apache.phoenix.parse.AliasedNode;
 import org.apache.phoenix.parse.BindTableNode;
 import org.apache.phoenix.parse.ColumnDef;
-import org.apache.phoenix.parse.CreateFunctionStatement;
 import org.apache.phoenix.parse.CreateTableStatement;
 import org.apache.phoenix.parse.DMLStatement;
 import org.apache.phoenix.parse.DerivedTableNode;
 import org.apache.phoenix.parse.FamilyWildcardParseNode;
 import org.apache.phoenix.parse.JoinTableNode;
-import org.apache.phoenix.parse.NamedNode;
 import org.apache.phoenix.parse.NamedTableNode;
 import org.apache.phoenix.parse.PFunction;
 import org.apache.phoenix.parse.ParseNode;
@@ -106,7 +101,7 @@ public class FromCompiler {
 
         @Override
         public List<TableRef> getTables() {
-            return Collections.emptyList();
+            return Collections.singletonList(TableRef.EMPTY_TABLE_REF);
         }
 
         @Override
@@ -117,16 +112,16 @@ public class FromCompiler {
         @Override
         public TableRef resolveTable(String schemaName, String tableName)
                 throws SQLException {
-            throw new UnsupportedOperationException();
+            throw new TableNotFoundException(schemaName, tableName);
         }
 
         @Override
         public ColumnRef resolveColumn(String schemaName, String tableName, String colName) throws SQLException {
-            throw new UnsupportedOperationException();
+            throw new ColumnNotFoundException(schemaName, tableName, null, colName);
         }
         
         public PFunction resolveFunction(String functionName) throws SQLException {
-            throw new UnsupportedOperationException();
+            throw new FunctionNotFoundException(functionName);
         };
 
         public boolean hasUDFs() {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/f3056ba6/phoenix-core/src/main/java/org/apache/phoenix/compile/ProjectionCompiler.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/ProjectionCompiler.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/ProjectionCompiler.java
index c39db09..915a55d 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/compile/ProjectionCompiler.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/ProjectionCompiler.java
@@ -407,7 +407,7 @@ public class ProjectionCompiler {
                 String columnAlias = aliasedNode.getAlias() != null ? aliasedNode.getAlias() : SchemaUtil.normalizeIdentifier(aliasedNode.getNode().getAlias());
                 boolean isCaseSensitive = aliasedNode.getAlias() != null ? aliasedNode.isCaseSensitve() : (columnAlias != null ? SchemaUtil.isCaseSensitive(aliasedNode.getNode().getAlias()) : selectVisitor.isCaseSensitive);
                 String name = columnAlias == null ? expression.toString() : columnAlias;
-                projectedColumns.add(new ExpressionProjector(name, tableRef.getTableAlias() == null ? table.getName().getString() : tableRef.getTableAlias(), expression, isCaseSensitive));
+                projectedColumns.add(new ExpressionProjector(name, tableRef.getTableAlias() == null ? (table.getName() == null ? "" : table.getName().getString()) : tableRef.getTableAlias(), expression, isCaseSensitive));
             }
             if(arrayKVFuncs.size() > 0 && arrayKVRefs.size() > 0) {
                 serailizeArrayIndexInformationAndSetInScan(context, arrayKVFuncs, arrayKVRefs);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/f3056ba6/phoenix-core/src/main/java/org/apache/phoenix/compile/QueryCompiler.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/QueryCompiler.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/QueryCompiler.java
index c02413a..83c6fd4 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/compile/QueryCompiler.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/QueryCompiler.java
@@ -35,6 +35,7 @@ import org.apache.phoenix.compile.OrderByCompiler.OrderBy;
 import org.apache.phoenix.execute.AggregatePlan;
 import org.apache.phoenix.execute.ClientAggregatePlan;
 import org.apache.phoenix.execute.ClientScanPlan;
+import org.apache.phoenix.execute.EmptyTableQueryPlan;
 import org.apache.phoenix.execute.HashJoinPlan;
 import org.apache.phoenix.execute.HashJoinPlan.HashSubPlan;
 import org.apache.phoenix.execute.HashJoinPlan.WhereClauseSubPlan;
@@ -555,9 +556,11 @@ public class QueryCompiler {
         QueryPlan plan = innerPlan;
         if (plan == null) {
             ParallelIteratorFactory parallelIteratorFactory = asSubquery ? null : this.parallelIteratorFactory;
-            plan = select.isAggregate() || select.isDistinct() ? 
-                      new AggregatePlan(context, select, tableRef, projector, limit, orderBy, parallelIteratorFactory, groupBy, having)
-                    : new ScanPlan(context, select, tableRef, projector, limit, orderBy, parallelIteratorFactory, allowPageFilter);
+            plan = select.getFrom() == null ?
+                      new EmptyTableQueryPlan(context, select, tableRef, projector, limit, orderBy, parallelIteratorFactory)
+                    : (select.isAggregate() || select.isDistinct() ? 
+                              new AggregatePlan(context, select, tableRef, projector, limit, orderBy, parallelIteratorFactory, groupBy, having)
+                            : new ScanPlan(context, select, tableRef, projector, limit, orderBy, parallelIteratorFactory, allowPageFilter));
         }
         if (!subqueries.isEmpty()) {
             int count = subqueries.size();

http://git-wip-us.apache.org/repos/asf/phoenix/blob/f3056ba6/phoenix-core/src/main/java/org/apache/phoenix/compile/TupleProjectionCompiler.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/TupleProjectionCompiler.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/TupleProjectionCompiler.java
index 5114411..ee65280 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/compile/TupleProjectionCompiler.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/TupleProjectionCompiler.java
@@ -61,6 +61,7 @@ public class TupleProjectionCompiler {
         Preconditions.checkArgument(!select.isJoin());
         // Non-group-by or group-by aggregations will create its own projected result.
         if (select.getInnerSelectStatement() != null 
+                || select.getFrom() == null
                 || select.isAggregate() 
                 || select.isDistinct()
                 || (context.getResolver().getTables().get(0).getTable().getType() != PTableType.TABLE

http://git-wip-us.apache.org/repos/asf/phoenix/blob/f3056ba6/phoenix-core/src/main/java/org/apache/phoenix/execute/BaseQueryPlan.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/execute/BaseQueryPlan.java b/phoenix-core/src/main/java/org/apache/phoenix/execute/BaseQueryPlan.java
index 25566d5..480be64 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/execute/BaseQueryPlan.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/execute/BaseQueryPlan.java
@@ -17,9 +17,6 @@
  */
 package org.apache.phoenix.execute;
 
-import static org.apache.phoenix.compile.OrderByCompiler.OrderBy.FWD_ROW_KEY_ORDER_BY;
-import static org.apache.phoenix.compile.OrderByCompiler.OrderBy.REV_ROW_KEY_ORDER_BY;
-
 import java.io.ByteArrayOutputStream;
 import java.io.DataOutputStream;
 import java.io.IOException;
@@ -56,8 +53,6 @@ import org.apache.phoenix.parse.FilterableStatement;
 import org.apache.phoenix.parse.HintNode.Hint;
 import org.apache.phoenix.parse.ParseNodeFactory;
 import org.apache.phoenix.parse.TableName;
-import org.apache.phoenix.query.QueryServices;
-import org.apache.phoenix.query.QueryServicesOptions;
 import org.apache.phoenix.schema.KeyValueSchema;
 import org.apache.phoenix.schema.PColumn;
 import org.apache.phoenix.schema.PName;
@@ -170,6 +165,10 @@ public abstract class BaseQueryPlan implements QueryPlan {
             return ResultIterator.EMPTY_ITERATOR;
         }
         
+        if (tableRef == TableRef.EMPTY_TABLE_REF) {
+            return newIterator(scanGrouper);
+        }
+        
         // Set miscellaneous scan attributes. This is the last chance to set them before we
         // clone the scan for each parallelized chunk.
         Scan scan = context.getScan();

http://git-wip-us.apache.org/repos/asf/phoenix/blob/f3056ba6/phoenix-core/src/main/java/org/apache/phoenix/execute/EmptyTableQueryPlan.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/execute/EmptyTableQueryPlan.java b/phoenix-core/src/main/java/org/apache/phoenix/execute/EmptyTableQueryPlan.java
new file mode 100644
index 0000000..fd55732
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/execute/EmptyTableQueryPlan.java
@@ -0,0 +1,96 @@
+/*
+ * 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.execute;
+
+import java.sql.SQLException;
+import java.util.Collections;
+import java.util.List;
+
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.phoenix.compile.GroupByCompiler.GroupBy;
+import org.apache.phoenix.compile.OrderByCompiler.OrderBy;
+import org.apache.phoenix.compile.RowProjector;
+import org.apache.phoenix.compile.StatementContext;
+import org.apache.phoenix.iterate.ParallelIteratorFactory;
+import org.apache.phoenix.iterate.ParallelScanGrouper;
+import org.apache.phoenix.iterate.ResultIterator;
+import org.apache.phoenix.iterate.SequenceResultIterator;
+import org.apache.phoenix.parse.FilterableStatement;
+import org.apache.phoenix.query.KeyRange;
+import org.apache.phoenix.schema.TableRef;
+import org.apache.phoenix.schema.tuple.SingleKeyValueTuple;
+import org.apache.phoenix.schema.tuple.Tuple;
+
+public class EmptyTableQueryPlan extends BaseQueryPlan {
+
+    public EmptyTableQueryPlan(StatementContext context, FilterableStatement statement, 
+            TableRef tableRef, RowProjector projection, Integer limit, OrderBy orderBy,
+            ParallelIteratorFactory parallelIteratorFactory) {
+        super(context, statement, tableRef, projection, context.getBindManager().getParameterMetaData(), limit, orderBy, GroupBy.EMPTY_GROUP_BY, parallelIteratorFactory);
+    }
+
+    @Override
+    public List<KeyRange> getSplits() {
+        return Collections.emptyList();
+    }
+
+    @Override
+    public List<List<Scan>> getScans() {
+        return Collections.emptyList();
+    }
+
+    @Override
+    public boolean useRoundRobinIterator() throws SQLException {
+        return false;
+    }
+
+    @Override
+    protected ResultIterator newIterator(ParallelScanGrouper scanGrouper)
+            throws SQLException {
+        ResultIterator scanner = new ResultIterator() {
+            private boolean hasNext = true;
+
+            @Override
+            public void close() throws SQLException {
+                this.hasNext = false;
+            }
+
+            @Override
+            public Tuple next() throws SQLException {
+                if (hasNext) {
+                    hasNext = false;
+                    return new SingleKeyValueTuple(KeyValue.LOWESTKEY);
+                }
+                return null;
+            }
+
+            @Override
+            public void explain(List<String> planSteps) {
+            }
+            
+        };
+        
+        if (context.getSequenceManager().getSequenceCount() > 0) {
+            scanner = new SequenceResultIterator(scanner, context.getSequenceManager());
+        }
+        
+        return scanner;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/f3056ba6/phoenix-core/src/main/java/org/apache/phoenix/parse/SelectStatement.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/SelectStatement.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/SelectStatement.java
index 362e98d..93c7364 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/parse/SelectStatement.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/SelectStatement.java
@@ -124,8 +124,10 @@ public class SelectStatement implements FilterableStatement {
             buf.append(',');
         }
         buf.setLength(buf.length()-1);
-        buf.append(" FROM ");
-        fromTable.toSQL(resolver, buf);
+        if (fromTable != null) {
+            buf.append(" FROM ");
+            fromTable.toSQL(resolver, buf);
+        }
         if (where != null) {
             buf.append(" WHERE ");
             where.toSQL(resolver, buf);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/f3056ba6/phoenix-core/src/main/java/org/apache/phoenix/schema/PTableImpl.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/PTableImpl.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/PTableImpl.java
index 4650739..aa9ee41 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/PTableImpl.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/PTableImpl.java
@@ -130,6 +130,7 @@ public class PTableImpl implements PTable {
     public PTableImpl() {
         this.indexes = Collections.emptyList();
         this.physicalNames = Collections.emptyList();
+        this.rowKeySchema = RowKeySchema.EMPTY_SCHEMA;
     }
 
     public PTableImpl(PName tenantId, String schemaName, String tableName, long timestamp, List<PColumnFamily> families) { // For base table of mapped VIEW

http://git-wip-us.apache.org/repos/asf/phoenix/blob/f3056ba6/phoenix-core/src/main/java/org/apache/phoenix/schema/TableRef.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/TableRef.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/TableRef.java
index bd88770..78b00fa 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/TableRef.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/TableRef.java
@@ -26,6 +26,8 @@ import org.apache.phoenix.util.SchemaUtil;
 
 
 public class TableRef {
+    public static final TableRef EMPTY_TABLE_REF = new TableRef(new PTableImpl());
+    
     private PTable table;
     private final String alias;
     private final long upperBoundTimeStamp;

http://git-wip-us.apache.org/repos/asf/phoenix/blob/f3056ba6/phoenix-core/src/test/java/org/apache/phoenix/compile/QueryCompilerTest.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/compile/QueryCompilerTest.java b/phoenix-core/src/test/java/org/apache/phoenix/compile/QueryCompilerTest.java
index 7f15329..6e637b7 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/compile/QueryCompilerTest.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/compile/QueryCompilerTest.java
@@ -1900,5 +1900,50 @@ public class QueryCompilerTest extends BaseConnectionlessQueryTest {
             conn.close();
         }
     }
+     
+     @Test
+     public void testNoFromClauseSelect() throws Exception {
+         Connection conn = DriverManager.getConnection(getUrl());
+         ResultSet rs = conn.createStatement().executeQuery("SELECT 2 * 3 * 4, 5 + 1");
+         assertTrue(rs.next());
+         assertEquals(24, rs.getInt(1));
+         assertEquals(6, rs.getInt(2));
+         assertFalse(rs.next());
+         
+         String query = 
+                 "SELECT 'a' AS col\n" +
+                 "UNION ALL\n" +
+                 "SELECT 'b' AS col\n" +
+                 "UNION ALL\n" +
+                 "SELECT 'c' AS col";
+         rs = conn.createStatement().executeQuery(query);
+         assertTrue(rs.next());
+         assertEquals("a", rs.getString(1));
+         assertTrue(rs.next());
+         assertEquals("b", rs.getString(1));
+         assertTrue(rs.next());
+         assertEquals("c", rs.getString(1));
+         assertFalse(rs.next());
+ 
+         rs = conn.createStatement().executeQuery("SELECT * FROM (" + query + ")");
+         assertTrue(rs.next());
+         assertEquals("a", rs.getString(1));
+         assertTrue(rs.next());
+         assertEquals("b", rs.getString(1));
+         assertTrue(rs.next());
+         assertEquals("c", rs.getString(1));
+         assertFalse(rs.next());
+     }
+     
+     
+     @Test
+     public void testFailNoFromClauseSelect() throws Exception {
+         Connection conn = DriverManager.getConnection(getUrl());
+         try {
+             conn.createStatement().executeQuery("SELECT foo, bar");
+             fail("Should have got ColumnNotFoundException");
+         } catch (ColumnNotFoundException e) {            
+         }
+     }
 
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/f3056ba6/phoenix-core/src/test/java/org/apache/phoenix/parse/QueryParserTest.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/parse/QueryParserTest.java b/phoenix-core/src/test/java/org/apache/phoenix/parse/QueryParserTest.java
index eabd789..d651246 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/parse/QueryParserTest.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/parse/QueryParserTest.java
@@ -212,7 +212,7 @@ public class QueryParserTest {
             parseQuery(sql);
             fail();
         } catch (SQLException e) {
-            assertEquals(SQLExceptionCode.MISMATCHED_TOKEN.getErrorCode(), e.getErrorCode());
+            assertEquals(SQLExceptionCode.MISSING_TOKEN.getErrorCode(), e.getErrorCode());
         }
     }
 
@@ -348,7 +348,7 @@ public class QueryParserTest {
             parseQuery(sql);
             fail("Should have caught exception.");
         } catch (SQLException e) {
-            assertTrue(e.getMessage(), e.getMessage().contains("ERROR 602 (42P00): Syntax error. Missing \"FROM\" at line 1, column 16."));
+            assertTrue(e.getMessage(), e.getMessage().contains("ERROR 602 (42P00): Syntax error. Missing \"EOF\" at line 1, column 16."));
         }
     }
 
@@ -375,19 +375,6 @@ public class QueryParserTest {
     }
 
     @Test
-    public void testParsingStatementWithMissingToken() throws Exception {
-        try {
-            String sql = ((
-                    "select a b\n" +
-                    "where e = d\n"));
-            parseQuery(sql);
-            fail("Should have caught exception.");
-        } catch (SQLException e) {
-            assertEquals(SQLExceptionCode.MISMATCHED_TOKEN.getErrorCode(), e.getErrorCode());
-        }
-    }
-
-    @Test
     public void testParseCreateTableInlinePrimaryKeyWithOrder() throws Exception {
     	for (String order : new String[]{"asc", "desc"}) {
             String s = "create table core.entity_history_archive (id char(15) primary key ${o})".replace("${o}", order);
@@ -612,7 +599,7 @@ public class QueryParserTest {
             parseQuery(sql);
             fail();
         } catch (SQLException e) {
-            assertEquals(SQLExceptionCode.MISMATCHED_TOKEN.getErrorCode(), e.getErrorCode());
+            assertEquals(SQLExceptionCode.MISSING_TOKEN.getErrorCode(), e.getErrorCode());
         }
     }