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 2014/12/11 00:06:34 UTC

phoenix git commit: PHOENIX-1489 Access column values positionally from client

Repository: phoenix
Updated Branches:
  refs/heads/calcite b005b1246 -> a2ee0efc9


PHOENIX-1489 Access column values positionally from client


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

Branch: refs/heads/calcite
Commit: a2ee0efc9aa07940de6affacca67dd7b937504d2
Parents: b005b12
Author: maryannxue <ma...@apache.org>
Authored: Wed Dec 10 18:06:14 2014 -0500
Committer: maryannxue <ma...@apache.org>
Committed: Wed Dec 10 18:06:14 2014 -0500

----------------------------------------------------------------------
 .../org/apache/phoenix/calcite/CalciteTest.java |   2 +-
 .../apache/phoenix/calcite/PhoenixProject.java  |   2 +-
 .../org/apache/phoenix/calcite/PhoenixRel.java  |   3 +
 .../calcite/PhoenixRelImplementorImpl.java      | 155 +++++++++++++------
 .../apache/phoenix/compile/FromCompiler.java    |   5 +-
 .../apache/phoenix/compile/JoinCompiler.java    |   4 +-
 .../apache/phoenix/compile/OrderByCompiler.java |   3 +-
 .../TrackOrderPreservingExpressionCompiler.java |   2 +-
 .../apache/phoenix/compile/UpsertCompiler.java  |   2 +-
 .../apache/phoenix/compile/WhereCompiler.java   |   2 +-
 .../org/apache/phoenix/schema/ColumnRef.java    |   2 +-
 .../org/apache/phoenix/schema/PTableImpl.java   |   8 +
 .../org/apache/phoenix/schema/PTableType.java   |   3 +-
 .../org/apache/phoenix/schema/TableRef.java     |   2 +-
 14 files changed, 132 insertions(+), 63 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/a2ee0efc/phoenix-core/src/it/java/org/apache/phoenix/calcite/CalciteTest.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/calcite/CalciteTest.java b/phoenix-core/src/it/java/org/apache/phoenix/calcite/CalciteTest.java
index 9bce0a3..a723303 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/calcite/CalciteTest.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/calcite/CalciteTest.java
@@ -182,7 +182,7 @@ public class CalciteTest extends BaseClientManagedTimeIT {
             new PhoenixSchema(phoenixConnection));
         calciteConnection.setSchema("phoenix");
         final Statement statement = calciteConnection.createStatement();
-        final ResultSet resultSet = statement.executeQuery("select * from aTable where a_string = 'a'");
+        final ResultSet resultSet = statement.executeQuery("select organization_id, entity_id, a_string from aTable where a_string = 'a'");
         while (resultSet.next()) {
             System.out.println("org_id=" + resultSet.getObject(1) + ",entity_id=" + resultSet.getObject(2) + ",a_string=" + resultSet.getObject("A_STRING"));
         }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/a2ee0efc/phoenix-core/src/main/java/org/apache/phoenix/calcite/PhoenixProject.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/calcite/PhoenixProject.java b/phoenix-core/src/main/java/org/apache/phoenix/calcite/PhoenixProject.java
index 2e90397..aa37471 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/calcite/PhoenixProject.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/calcite/PhoenixProject.java
@@ -34,7 +34,7 @@ public class PhoenixProject extends Project implements PhoenixRel {
 
     @Override
     public void implement(Implementor implementor, PhoenixConnection conn) {
+    	implementor.setProjects(getProjects());
         implementor.visitInput(0, (PhoenixRel) getInput());
-        throw new UnsupportedOperationException();
     }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/a2ee0efc/phoenix-core/src/main/java/org/apache/phoenix/calcite/PhoenixRel.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/calcite/PhoenixRel.java b/phoenix-core/src/main/java/org/apache/phoenix/calcite/PhoenixRel.java
index c255e90..9bd4d70 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/calcite/PhoenixRel.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/calcite/PhoenixRel.java
@@ -1,5 +1,7 @@
 package org.apache.phoenix.calcite;
 
+import java.util.List;
+
 import org.apache.calcite.plan.Convention;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rex.RexNode;
@@ -34,6 +36,7 @@ public interface PhoenixRel extends RelNode {
     void visitInput(int i, PhoenixRel input);
     ColumnExpression newColumnExpression(int index);
     void setContext(PhoenixConnection conn, PTable pTable, RexNode filter);
+    void setProjects(List<? extends RexNode> projects);
     QueryPlan makePlan();
   }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/a2ee0efc/phoenix-core/src/main/java/org/apache/phoenix/calcite/PhoenixRelImplementorImpl.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/calcite/PhoenixRelImplementorImpl.java b/phoenix-core/src/main/java/org/apache/phoenix/calcite/PhoenixRelImplementorImpl.java
index a833b6b..2fa7849 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/calcite/PhoenixRelImplementorImpl.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/calcite/PhoenixRelImplementorImpl.java
@@ -1,7 +1,9 @@
 package org.apache.phoenix.calcite;
 
 import java.sql.SQLException;
+import java.util.ArrayList;
 import java.util.Collections;
+import java.util.List;
 
 import org.apache.calcite.rex.RexNode;
 import org.apache.hadoop.hbase.client.Scan;
@@ -16,80 +18,139 @@ import org.apache.phoenix.compile.StatementContext;
 import org.apache.phoenix.compile.GroupByCompiler.GroupBy;
 import org.apache.phoenix.compile.WhereCompiler;
 import org.apache.phoenix.compile.WhereOptimizer;
-import org.apache.phoenix.execute.AggregatePlan;
 import org.apache.phoenix.execute.ScanPlan;
+import org.apache.phoenix.execute.TupleProjectionPlan;
+import org.apache.phoenix.execute.TupleProjector;
 import org.apache.phoenix.expression.ColumnExpression;
 import org.apache.phoenix.expression.Expression;
 import org.apache.phoenix.iterate.ParallelIteratorFactory;
 import org.apache.phoenix.jdbc.PhoenixConnection;
 import org.apache.phoenix.jdbc.PhoenixStatement;
+import org.apache.phoenix.parse.AliasedNode;
 import org.apache.phoenix.parse.ColumnDef;
-import org.apache.phoenix.parse.FilterableStatement;
 import org.apache.phoenix.parse.NamedTableNode;
 import org.apache.phoenix.parse.SelectStatement;
 import org.apache.phoenix.parse.TableName;
+import org.apache.phoenix.parse.TableWildcardParseNode;
 import org.apache.phoenix.schema.ColumnRef;
+import org.apache.phoenix.schema.PColumn;
+import org.apache.phoenix.schema.PColumnImpl;
 import org.apache.phoenix.schema.PDatum;
+import org.apache.phoenix.schema.PNameFactory;
 import org.apache.phoenix.schema.PTable;
+import org.apache.phoenix.schema.PTableImpl;
+import org.apache.phoenix.schema.PTableType;
 import org.apache.phoenix.schema.TableRef;
+import org.apache.phoenix.schema.KeyValueSchema.KeyValueSchemaBuilder;
+import org.apache.phoenix.util.SchemaUtil;
 
 import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Lists;
 
 class PhoenixRelImplementorImpl implements PhoenixRel.Implementor {
-	private TableRef tableRef;
-	private PhoenixConnection conn;
-	private StatementContext context;
-	private RowProjector projector;
-	private SelectStatement select;
-	
+    private TableRef tableRef;
+    private PhoenixConnection conn;
+    private StatementContext context;
+    private RowProjector projector;
+    private SelectStatement select;
+    private List<? extends RexNode> projects;
+    private List<Expression> projectExpressions;
+    
     @Override
     public void visitInput(int i, PhoenixRel input) {
         input.implement(this, conn);
     }
 
-	@Override
-	public ColumnExpression newColumnExpression(int index) {
-		ColumnRef colRef = new ColumnRef(tableRef, index);
-		return colRef.newColumnExpression();
-	}
+    @Override
+    public ColumnExpression newColumnExpression(int index) {
+        ColumnRef colRef = new ColumnRef(tableRef, index);
+        return colRef.newColumnExpression();
+    }
 
 
-	@Override
-	public void setContext(PhoenixConnection conn, PTable table, RexNode filter) {
-		this.conn = conn;
-		this.tableRef = new TableRef(table);
-		PhoenixStatement stmt = new PhoenixStatement(conn);
+    @Override
+    public void setContext(PhoenixConnection conn, PTable table, RexNode filter) {
+        this.conn = conn;
+        this.tableRef = new TableRef(table);
+        PhoenixStatement stmt = new PhoenixStatement(conn);
         ColumnResolver resolver;
-		try {
-			resolver = FromCompiler.getResolver(
-			        NamedTableNode.create(
-			            null,
-			            TableName.create(tableRef.getTable().getSchemaName().getString(), tableRef.getTable().getTableName().getString()),
-			            ImmutableList.<ColumnDef>of()), conn);
-	        this.context = new StatementContext(stmt, resolver, new Scan(), new SequenceManager(stmt));
-	        // TODO: real projection
-	        this.select = SelectStatement.SELECT_STAR;
-	        this.projector = ProjectionCompiler.compile(context, select, GroupBy.EMPTY_GROUP_BY, Collections.<PDatum>emptyList());
-	        if (filter != null) {
-	        	Expression filterExpr = CalciteUtils.toExpression(filter, this);
-	        	filterExpr = WhereOptimizer.pushKeyExpressionsToScan(context, select, filterExpr);
-	        	WhereCompiler.setScanFilter(context, select, filterExpr, true, false);
-	        }
-		} catch (SQLException e) {
-			throw new RuntimeException(e);
-		}
-	}
+        try {
+            resolver = FromCompiler.getResolver(
+                    NamedTableNode.create(
+                        null,
+                        TableName.create(tableRef.getTable().getSchemaName().getString(), tableRef.getTable().getTableName().getString()),
+                        ImmutableList.<ColumnDef>of()), conn);
+            this.context = new StatementContext(stmt, resolver, new Scan(), new SequenceManager(stmt));
+            if (filter != null) {
+                Expression filterExpr = CalciteUtils.toExpression(filter, this);
+                filterExpr = WhereOptimizer.pushKeyExpressionsToScan(context, select, filterExpr);
+                WhereCompiler.setScanFilter(context, select, filterExpr, true, false);
+            }
+            this.projectExpressions = Lists.<Expression> newArrayListWithExpectedSize(projects.size());
+            for (RexNode p : projects) {
+                this.projectExpressions.add(CalciteUtils.toExpression(p, this));
+            }
+            PTable projectedTable = createProjectedTable(table);
+            this.context.setResolver(FromCompiler.getResolver(new TableRef(projectedTable)));
+            this.select = SelectStatement.create(SelectStatement.SELECT_STAR, Collections.<AliasedNode>singletonList(new AliasedNode(null, TableWildcardParseNode.create(TableName.create(table.getSchemaName().getString(), table.getName().getString()), false))));
+            this.projector = ProjectionCompiler.compile(context, select, GroupBy.EMPTY_GROUP_BY, Collections.<PDatum>emptyList());
+        } catch (SQLException e) {
+            throw new RuntimeException(e);
+        }
+    }
 
-	@Override
-	public QueryPlan makePlan() {
-            try {
-		Integer limit = null;
-		OrderBy orderBy = OrderBy.EMPTY_ORDER_BY;
-		ParallelIteratorFactory iteratorFactory = null;
-                return new ScanPlan(context, select, tableRef, projector, limit, orderBy, iteratorFactory, true);
-            } catch (SQLException e) {
-                throw new RuntimeException(e);
+    @Override
+    public void setProjects(List<? extends RexNode> projects) {
+        this.projects = projects;
+    }
+
+    @Override
+    public QueryPlan makePlan() {
+        try {
+            Integer limit = null;
+            OrderBy orderBy = OrderBy.EMPTY_ORDER_BY;
+            ParallelIteratorFactory iteratorFactory = null;
+            QueryPlan plan = new ScanPlan(context, select, tableRef, projector, limit, orderBy, iteratorFactory, true);
+            TupleProjector tupleProjector = createTupleProjector(tableRef);
+            return new TupleProjectionPlan(plan, tupleProjector, null);
+        } catch (SQLException e) {
+            throw new RuntimeException(e);
+        }
+    }
+    
+    private PTable createProjectedTable(PTable table) throws SQLException {
+        if (this.projects == null) {
+            return PTableImpl.makePTable(table, PTableType.PROJECTED, table.getColumns());
+        }
+        
+        List<PColumn> projectedColumns = new ArrayList<PColumn>();
+        for (int i = 0; i < this.projects.size(); i++) {
+            Expression e = this.projectExpressions.get(i);
+            PColumnImpl projectedColumn = new PColumnImpl(PNameFactory.newName(
+                    this.projects.get(i).toString()), PNameFactory.newName(TupleProjector.VALUE_COLUMN_FAMILY), 
+                    e.getDataType(), e.getMaxLength(), e.getScale(), e.isNullable(), 
+                    projectedColumns.size(), e.getSortOrder(), null, null, false);                
+            projectedColumns.add(projectedColumn);
+        }
+        return PTableImpl.makePTable(table, PTableType.PROJECTED, projectedColumns);        
+    }
+    
+    private TupleProjector createTupleProjector(TableRef tableRef) {
+        KeyValueSchemaBuilder builder = new KeyValueSchemaBuilder(0);
+        List<Expression> exprs = this.projectExpressions;
+        if (exprs == null) {
+            exprs = Lists.<Expression> newArrayList();
+            for (PColumn column : tableRef.getTable().getColumns()) {
+                if (!SchemaUtil.isPKColumn(column)) {
+                    exprs.add(newColumnExpression(column.getPosition()));
+                }
             }
-	}
+        }
+        for (Expression e : exprs) {
+            builder.addField(e);                
+        }
+        
+        return new TupleProjector(builder.build(), exprs.toArray(new Expression[exprs.size()]));
+    }
 
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/a2ee0efc/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 2c4a578..6b88c2e 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
@@ -24,7 +24,6 @@ import java.util.Collections;
 import java.util.Iterator;
 import java.util.List;
 
-import com.google.common.base.Preconditions;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.client.HTableInterface;
 import org.apache.phoenix.coprocessor.MetaDataProtocol;
@@ -318,7 +317,7 @@ public class FromCompiler {
         private final int tsAddition;
         
         private BaseColumnResolver(PhoenixConnection connection, int tsAddition) {
-            this.connection = Preconditions.checkNotNull(connection);
+            this.connection = connection;
             this.client = connection == null ? null : new MetaDataClient(connection);
             this.tsAddition = tsAddition;
         }
@@ -474,7 +473,7 @@ public class FromCompiler {
                 columns.add(column);
             }
             PTable t = PTableImpl.makePTable(null, PName.EMPTY_NAME, PName.EMPTY_NAME, 
-                    PTableType.SUBQUERY, null, MetaDataProtocol.MIN_TABLE_TIMESTAMP, PTable.INITIAL_SEQ_NUM, 
+                    PTableType.PROJECTED, null, MetaDataProtocol.MIN_TABLE_TIMESTAMP, PTable.INITIAL_SEQ_NUM, 
                     null, null, columns, null, null, Collections.<PTable>emptyList(), 
                     false, Collections.<PName>emptyList(), null, null, false, false, null, null, null);
             

http://git-wip-us.apache.org/repos/asf/phoenix/blob/a2ee0efc/phoenix-core/src/main/java/org/apache/phoenix/compile/JoinCompiler.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/JoinCompiler.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/JoinCompiler.java
index 45b6603..0beddef 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/compile/JoinCompiler.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/JoinCompiler.java
@@ -739,7 +739,7 @@ public class JoinCompiler {
                 }               
             }
             
-            PTable t = PTableImpl.makePTable(table.getTenantId(), PNameFactory.newName(PROJECTED_TABLE_SCHEMA), table.getName(), PTableType.JOIN,
+            PTable t = PTableImpl.makePTable(table.getTenantId(), PNameFactory.newName(PROJECTED_TABLE_SCHEMA), table.getName(), PTableType.PROJECTED,
                         table.getIndexState(), table.getTimeStamp(), table.getSequenceNumber(), table.getPKName(),
                         retainPKColumns ? table.getBucketNum() : null, projectedColumns, table.getParentSchemaName(),
                         table.getParentTableName(), table.getIndexes(), table.isImmutableRows(), Collections.<PName>emptyList(), null, null, table.isWALDisabled(), table.isMultiTenant(), table.getViewType(), table.getViewIndexId(), table.getIndexType());
@@ -789,7 +789,7 @@ public class JoinCompiler {
                 projectedColumns.add(projectedColumn);
                 sourceExpressions.add(sourceExpression);
             }
-            PTable t = PTableImpl.makePTable(table.getTenantId(), PNameFactory.newName(PROJECTED_TABLE_SCHEMA), table.getName(), PTableType.JOIN,
+            PTable t = PTableImpl.makePTable(table.getTenantId(), PNameFactory.newName(PROJECTED_TABLE_SCHEMA), table.getName(), PTableType.PROJECTED,
                         table.getIndexState(), table.getTimeStamp(), table.getSequenceNumber(), table.getPKName(),
                         null, projectedColumns, table.getParentSchemaName(),
                         table.getParentTableName(), table.getIndexes(), table.isImmutableRows(), Collections.<PName>emptyList(), null, null, table.isWALDisabled(), table.isMultiTenant(), table.getViewType(), table.getViewIndexId(), table.getIndexType());

http://git-wip-us.apache.org/repos/asf/phoenix/blob/a2ee0efc/phoenix-core/src/main/java/org/apache/phoenix/compile/OrderByCompiler.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/OrderByCompiler.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/OrderByCompiler.java
index 444b05e..bb3f324 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/compile/OrderByCompiler.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/OrderByCompiler.java
@@ -123,8 +123,7 @@ public class OrderByCompiler {
                 // REV_ROW_KEY_ORDER_BY scan would not take effect for a projected table, so don't return it for such table types.
                 if (context.getConnection().getQueryServices().getProps().getBoolean(QueryServices.USE_REVERSE_SCAN_ATTRIB, QueryServicesOptions.DEFAULT_USE_REVERSE_SCAN)
                         && !context.getScanRanges().useSkipScanFilter()
-                        && context.getCurrentTable().getTable().getType() != PTableType.JOIN
-                        && context.getCurrentTable().getTable().getType() != PTableType.SUBQUERY) {
+                        && context.getCurrentTable().getTable().getType() != PTableType.PROJECTED) {
                     return OrderBy.REV_ROW_KEY_ORDER_BY;
                 }
             } else {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/a2ee0efc/phoenix-core/src/main/java/org/apache/phoenix/compile/TrackOrderPreservingExpressionCompiler.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/TrackOrderPreservingExpressionCompiler.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/TrackOrderPreservingExpressionCompiler.java
index 9fd6837..3bd2cc8 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/compile/TrackOrderPreservingExpressionCompiler.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/TrackOrderPreservingExpressionCompiler.java
@@ -177,7 +177,7 @@ public class TrackOrderPreservingExpressionCompiler extends ExpressionCompiler {
     }
     
     private int getColumnPKPosition(ColumnRef ref) {
-        if (tupleProjector != null && ref.getTable().getType() == PTableType.SUBQUERY) {
+        if (tupleProjector != null && ref.getTable().getType() == PTableType.PROJECTED) {
             Expression expression = tupleProjector.getExpressions()[ref.getColumnPosition()];
             if (expression instanceof RowKeyColumnExpression) {
                 return ((RowKeyColumnExpression) expression).getPosition();

http://git-wip-us.apache.org/repos/asf/phoenix/blob/a2ee0efc/phoenix-core/src/main/java/org/apache/phoenix/compile/UpsertCompiler.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/UpsertCompiler.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/UpsertCompiler.java
index 796f368..0bb4f1e 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/compile/UpsertCompiler.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/UpsertCompiler.java
@@ -421,7 +421,7 @@ public class UpsertCompiler {
                         queryPlanToBe = compiler.compile();
                         // This is post-fix: if the tableRef is a projected table, this means there are post-processing 
                         // steps and parallelIteratorFactory did not take effect.
-                        if (queryPlanToBe.getTableRef().getTable().getType() == PTableType.JOIN || queryPlanToBe.getTableRef().getTable().getType() == PTableType.SUBQUERY) {
+                        if (queryPlanToBe.getTableRef().getTable().getType() == PTableType.PROJECTED) {
                             parallelIteratorFactoryToBe = null;
                         }
                     } catch (MetaDataEntityNotFoundException e) {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/a2ee0efc/phoenix-core/src/main/java/org/apache/phoenix/compile/WhereCompiler.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/WhereCompiler.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/WhereCompiler.java
index 694eb6a..43202ea 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/compile/WhereCompiler.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/WhereCompiler.java
@@ -145,7 +145,7 @@ public class WhereCompiler {
             expression = AndExpression.create(filters);
         }
         
-        if (context.getCurrentTable().getTable().getType() != PTableType.JOIN && context.getCurrentTable().getTable().getType() != PTableType.SUBQUERY) {
+        if (context.getCurrentTable().getTable().getType() != PTableType.PROJECTED) {
             expression = WhereOptimizer.pushKeyExpressionsToScan(context, statement, expression, extractedNodes);
         }
         setScanFilter(context, statement, expression, whereCompiler.disambiguateWithFamily, hashJoinOptimization);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/a2ee0efc/phoenix-core/src/main/java/org/apache/phoenix/schema/ColumnRef.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/ColumnRef.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/ColumnRef.java
index f271ac5..3112ad8 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/ColumnRef.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/ColumnRef.java
@@ -101,7 +101,7 @@ public class ColumnRef {
                     displayName);
         }
         
-        if (table.getType() == PTableType.JOIN || table.getType() == PTableType.SUBQUERY) {
+        if (table.getType() == PTableType.PROJECTED) {
         	return new ProjectedColumnExpression(column, table, displayName);
         }
        

http://git-wip-us.apache.org/repos/asf/phoenix/blob/a2ee0efc/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 a877175..4f69895 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
@@ -197,6 +197,14 @@ public class PTableImpl implements PTable {
                 table.isWALDisabled(), table.isMultiTenant(), table.getViewType(), table.getViewIndexId(), table.getIndexType(), table.getTableStats());
     }
 
+    public static PTableImpl makePTable(PTable table, PTableType type, List<PColumn> columns) throws SQLException {
+        return new PTableImpl(
+                table.getTenantId(), table.getSchemaName(), table.getTableName(), type, table.getIndexState(), table.getTimeStamp(), 
+                table.getSequenceNumber(), table.getPKName(), table.getBucketNum(), columns, table.getParentSchemaName(), table.getParentTableName(),
+                table.getIndexes(), table.isImmutableRows(), table.getPhysicalNames(), table.getDefaultFamilyName(), table.getViewStatement(),
+                table.isWALDisabled(), table.isMultiTenant(), table.getViewType(), table.getViewIndexId(), table.getIndexType(), table.getTableStats());
+    }
+
     public static PTableImpl makePTable(PTable table, long timeStamp, long sequenceNumber, List<PColumn> columns) throws SQLException {
         return new PTableImpl(
                 table.getTenantId(), table.getSchemaName(), table.getTableName(), table.getType(), table.getIndexState(), timeStamp, 

http://git-wip-us.apache.org/repos/asf/phoenix/blob/a2ee0efc/phoenix-core/src/main/java/org/apache/phoenix/schema/PTableType.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/PTableType.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/PTableType.java
index 23ba829..ff3d0bb 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/PTableType.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/PTableType.java
@@ -27,8 +27,7 @@ public enum PTableType {
     TABLE("u", "TABLE"),
     VIEW("v", "VIEW"),
     INDEX("i", "INDEX"),
-    JOIN("j", "JOIN"),
-    SUBQUERY("q", "SUBQUERY"); 
+    PROJECTED("p", "PROJECTED"); 
 
     private final PName value;
     private final String serializedValue;

http://git-wip-us.apache.org/repos/asf/phoenix/blob/a2ee0efc/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 a88ba4d..9ea2c96 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
@@ -71,7 +71,7 @@ public class TableRef {
 
     public String getColumnDisplayName(ColumnRef ref) {
         PColumn column = ref.getColumn();
-        if (table.getType() == PTableType.JOIN || table.getType() == PTableType.SUBQUERY) {
+        if (table.getType() == PTableType.PROJECTED) {
             return column.getName().getString();
         }
         boolean isIndex = table.getType() == PTableType.INDEX;