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 2016/11/08 19:44:11 UTC

phoenix git commit: PHOENIX-3466 Add StatementContext instance in PhoenixImplementorImpl

Repository: phoenix
Updated Branches:
  refs/heads/calcite 605b25aff -> 75caf0906


PHOENIX-3466 Add StatementContext instance in PhoenixImplementorImpl


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

Branch: refs/heads/calcite
Commit: 75caf0906343a1a1603fa42ec11dd98322be91d5
Parents: 605b25a
Author: maryannxue <ma...@gmail.com>
Authored: Tue Nov 8 11:44:05 2016 -0800
Committer: maryannxue <ma...@gmail.com>
Committed: Tue Nov 8 11:44:05 2016 -0800

----------------------------------------------------------------------
 .../calcite/jdbc/PhoenixCalciteFactory.java     |  4 +-
 .../org/apache/calcite/sql/ListJarsTable.java   |  1 -
 .../org/apache/calcite/sql/SqlOptionNode.java   |  4 +-
 .../apache/phoenix/calcite/CalciteUtils.java    |  1 -
 .../phoenix/calcite/PhoenixPrepareImpl.java     | 18 ++++++++-
 .../phoenix/calcite/PhoenixScalarFunction.java  |  2 +
 .../apache/phoenix/calcite/TableMapping.java    |  1 -
 .../calcite/rel/PhoenixRelImplementor.java      |  2 +
 .../calcite/rel/PhoenixRelImplementorImpl.java  | 11 +++++-
 .../phoenix/calcite/rel/PhoenixTableScan.java   |  3 +-
 .../rel/PhoenixToEnumerableConverter.java       | 18 ++++++---
 .../calcite/rules/PhoenixConverterRules.java    | 40 +++++++++-----------
 .../phoenix/calcite/ToExpressionTest.java       |  6 ++-
 13 files changed, 71 insertions(+), 40 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/75caf090/phoenix-core/src/main/java/org/apache/calcite/jdbc/PhoenixCalciteFactory.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/calcite/jdbc/PhoenixCalciteFactory.java b/phoenix-core/src/main/java/org/apache/calcite/jdbc/PhoenixCalciteFactory.java
index bc9994b..faab40a 100644
--- a/phoenix-core/src/main/java/org/apache/calcite/jdbc/PhoenixCalciteFactory.java
+++ b/phoenix-core/src/main/java/org/apache/calcite/jdbc/PhoenixCalciteFactory.java
@@ -120,7 +120,8 @@ public class PhoenixCalciteFactory extends CalciteFactory {
             super(driver, factory, url, info, rootSchema, typeFactory);
         }
 
-        public CalciteStatement createStatement(String sql, int resultSetType, int resultSetConcurrency, int resultSetHoldability) throws SQLException {
+        @Override
+        public CalciteStatement createStatement(int resultSetType, int resultSetConcurrency, int resultSetHoldability) throws SQLException {
             try {
                 return super.createStatement(resultSetType, resultSetConcurrency, resultSetHoldability);
             } catch (SQLException e) {
@@ -160,6 +161,7 @@ public class PhoenixCalciteFactory extends CalciteFactory {
             return super.enumerable(handle, signature);
         }
 
+        @Override
         public void abort(final Executor executor) throws SQLException {
             call(new PhoenixConnectionCallable() {
                 @Override

http://git-wip-us.apache.org/repos/asf/phoenix/blob/75caf090/phoenix-core/src/main/java/org/apache/calcite/sql/ListJarsTable.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/calcite/sql/ListJarsTable.java b/phoenix-core/src/main/java/org/apache/calcite/sql/ListJarsTable.java
index db2b003..a73d591 100644
--- a/phoenix-core/src/main/java/org/apache/calcite/sql/ListJarsTable.java
+++ b/phoenix-core/src/main/java/org/apache/calcite/sql/ListJarsTable.java
@@ -32,7 +32,6 @@ import org.apache.calcite.rel.type.RelDataTypeFactory;
 import org.apache.calcite.schema.ScannableTable;
 import org.apache.calcite.schema.impl.AbstractTable;
 import org.apache.calcite.sql.type.SqlTypeName;
-import org.apache.calcite.util.Smalls.MazeTable;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.LocatedFileStatus;

http://git-wip-us.apache.org/repos/asf/phoenix/blob/75caf090/phoenix-core/src/main/java/org/apache/calcite/sql/SqlOptionNode.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/calcite/sql/SqlOptionNode.java b/phoenix-core/src/main/java/org/apache/calcite/sql/SqlOptionNode.java
index bcec18a..149e9ef 100644
--- a/phoenix-core/src/main/java/org/apache/calcite/sql/SqlOptionNode.java
+++ b/phoenix-core/src/main/java/org/apache/calcite/sql/SqlOptionNode.java
@@ -42,8 +42,8 @@ public class SqlOptionNode extends SqlNode {
             propertyName = key.names.get(1);
         }
 
-        PhoenixRelImplementor
-                implementor = new PhoenixRelImplementorImpl(RuntimeContext.EMPTY_CONTEXT);
+        PhoenixRelImplementor implementor =
+                new PhoenixRelImplementorImpl(null, RuntimeContext.EMPTY_CONTEXT);
         this.value = CalciteUtils.convertSqlLiteral(literal, implementor);
     }
 

http://git-wip-us.apache.org/repos/asf/phoenix/blob/75caf090/phoenix-core/src/main/java/org/apache/phoenix/calcite/CalciteUtils.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/calcite/CalciteUtils.java b/phoenix-core/src/main/java/org/apache/phoenix/calcite/CalciteUtils.java
index b312954..1ed650b 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/calcite/CalciteUtils.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/calcite/CalciteUtils.java
@@ -738,7 +738,6 @@ public class CalciteUtils {
             }
         });
         EXPRESSION_MAP.put(SqlKind.DEFAULT, new ExpressionFactory() {
-            @SuppressWarnings("rawtypes")
             @Override
             public Expression newExpression(RexNode node, PhoenixRelImplementor implementor) {
                 return null;

http://git-wip-us.apache.org/repos/asf/phoenix/blob/75caf090/phoenix-core/src/main/java/org/apache/phoenix/calcite/PhoenixPrepareImpl.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/calcite/PhoenixPrepareImpl.java b/phoenix-core/src/main/java/org/apache/phoenix/calcite/PhoenixPrepareImpl.java
index 8ac009e..1d9ca1e 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/calcite/PhoenixPrepareImpl.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/calcite/PhoenixPrepareImpl.java
@@ -18,6 +18,7 @@ import org.apache.calcite.plan.RelOptRule;
 import org.apache.calcite.prepare.CalcitePrepareImpl;
 import org.apache.calcite.prepare.Prepare.Materialization;
 import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.convert.ConverterRule;
 import org.apache.calcite.rel.logical.LogicalSort;
 import org.apache.calcite.rel.rules.JoinCommuteRule;
 import org.apache.calcite.rel.rules.SortProjectTransposeRule;
@@ -59,6 +60,7 @@ import org.apache.phoenix.calcite.parser.PhoenixParserImpl;
 import org.apache.phoenix.calcite.rel.PhoenixRel;
 import org.apache.phoenix.calcite.rel.PhoenixServerProject;
 import org.apache.phoenix.calcite.rel.PhoenixTemporarySort;
+import org.apache.phoenix.calcite.rules.PhoenixConverterRules.PhoenixToEnumerableConverterRule;
 import org.apache.phoenix.calcite.rules.PhoenixFilterScanMergeRule;
 import org.apache.phoenix.calcite.rules.PhoenixForwardTableScanRule;
 import org.apache.phoenix.calcite.rules.PhoenixJoinSingleValueAggregateMergeRule;
@@ -72,6 +74,7 @@ import org.apache.phoenix.compile.CreateIndexCompiler;
 import org.apache.phoenix.compile.CreateSequenceCompiler;
 import org.apache.phoenix.compile.CreateTableCompiler;
 import org.apache.phoenix.compile.MutationPlan;
+import org.apache.phoenix.compile.StatementContext;
 import org.apache.phoenix.expression.LiteralExpression;
 import org.apache.phoenix.jdbc.PhoenixConnection;
 import org.apache.phoenix.jdbc.PhoenixStatement;
@@ -158,7 +161,20 @@ public class PhoenixPrepareImpl extends CalcitePrepareImpl {
                 PhoenixTemporarySort.class,
                 PhoenixServerProject.class,
                 "PhoenixSortProjectTransposeRule"));
-        
+
+        final PhoenixConnection pc =
+                getPhoenixConnection(prepareContext.getRootSchema().plus());
+        try {
+            final StatementContext context =
+                    new StatementContext((PhoenixStatement) pc.createStatement());
+            ConverterRule[] rules = PhoenixToEnumerableConverterRule
+                    .createPhoenixToEnumerableConverterRules(context);
+            for (ConverterRule rule : rules) {
+                planner.addRule(rule);
+            }
+        } catch (SQLException e) {
+            throw new RuntimeException(e);
+        }
         for (RelOptRule rule : this.defaultConverterRules) {
             planner.addRule(rule);
         }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/75caf090/phoenix-core/src/main/java/org/apache/phoenix/calcite/PhoenixScalarFunction.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/calcite/PhoenixScalarFunction.java b/phoenix-core/src/main/java/org/apache/phoenix/calcite/PhoenixScalarFunction.java
index a65f851..3a82a49 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/calcite/PhoenixScalarFunction.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/calcite/PhoenixScalarFunction.java
@@ -39,6 +39,7 @@ import com.google.common.collect.Lists;
 
 public class PhoenixScalarFunction implements ScalarFunction, ImplementableFunction {
     private final PFunction functionInfo;
+    @SuppressWarnings("rawtypes")
     private final PDataType returnType;
     private final List<FunctionParameter> parameters;
     
@@ -58,6 +59,7 @@ public class PhoenixScalarFunction implements ScalarFunction, ImplementableFunct
                             return getArgumentName(arg.getArgPosition());
                         }
 
+                        @SuppressWarnings("rawtypes")
                         public RelDataType getType(RelDataTypeFactory typeFactory) {
                             PDataType dataType =
                                     arg.isArrayType() ? PDataType.fromTypeId(PDataType.sqlArrayType(SchemaUtil

http://git-wip-us.apache.org/repos/asf/phoenix/blob/75caf090/phoenix-core/src/main/java/org/apache/phoenix/calcite/TableMapping.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/calcite/TableMapping.java b/phoenix-core/src/main/java/org/apache/phoenix/calcite/TableMapping.java
index 216920f..e27843c 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/calcite/TableMapping.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/calcite/TableMapping.java
@@ -23,7 +23,6 @@ import org.apache.phoenix.compile.RowProjector;
 import org.apache.phoenix.compile.TupleProjectionCompiler;
 import org.apache.phoenix.coprocessor.BaseScannerRegionObserver;
 import org.apache.phoenix.execute.TupleProjector;
-import org.apache.phoenix.expression.ColumnExpression;
 import org.apache.phoenix.expression.Expression;
 import org.apache.phoenix.expression.LiteralExpression;
 import org.apache.phoenix.index.IndexMaintainer;

http://git-wip-us.apache.org/repos/asf/phoenix/blob/75caf090/phoenix-core/src/main/java/org/apache/phoenix/calcite/rel/PhoenixRelImplementor.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/calcite/rel/PhoenixRelImplementor.java b/phoenix-core/src/main/java/org/apache/phoenix/calcite/rel/PhoenixRelImplementor.java
index cdcd74f..4b698c7 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/calcite/rel/PhoenixRelImplementor.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/calcite/rel/PhoenixRelImplementor.java
@@ -8,6 +8,7 @@ import org.apache.phoenix.calcite.TableMapping;
 import org.apache.phoenix.compile.QueryPlan;
 import org.apache.phoenix.compile.SequenceManager;
 import org.apache.phoenix.compile.SequenceValueExpression;
+import org.apache.phoenix.compile.StatementContext;
 import org.apache.phoenix.execute.RuntimeContext;
 import org.apache.phoenix.execute.TupleProjector;
 import org.apache.phoenix.expression.Expression;
@@ -24,6 +25,7 @@ public interface PhoenixRelImplementor {
     @SuppressWarnings("rawtypes")
     Expression newFieldAccessExpression(String variableId, int index, PDataType type);
     SequenceValueExpression newSequenceExpression(PhoenixSequence seq, SequenceValueParseNode.Op op);
+    StatementContext getStatementContext();
     RuntimeContext getRuntimeContext();
     void setTableMapping(TableMapping tableMapping);
     TableMapping getTableMapping();

http://git-wip-us.apache.org/repos/asf/phoenix/blob/75caf090/phoenix-core/src/main/java/org/apache/phoenix/calcite/rel/PhoenixRelImplementorImpl.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/calcite/rel/PhoenixRelImplementorImpl.java b/phoenix-core/src/main/java/org/apache/phoenix/calcite/rel/PhoenixRelImplementorImpl.java
index d581ec0..901ea07 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/calcite/rel/PhoenixRelImplementorImpl.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/calcite/rel/PhoenixRelImplementorImpl.java
@@ -10,6 +10,7 @@ import org.apache.phoenix.calcite.TableMapping;
 import org.apache.phoenix.compile.QueryPlan;
 import org.apache.phoenix.compile.SequenceManager;
 import org.apache.phoenix.compile.SequenceValueExpression;
+import org.apache.phoenix.compile.StatementContext;
 import org.apache.phoenix.coprocessor.MetaDataProtocol;
 import org.apache.phoenix.execute.RuntimeContext;
 import org.apache.phoenix.execute.TupleProjector;
@@ -31,12 +32,15 @@ import org.apache.phoenix.schema.types.PDataType;
 import com.google.common.collect.Lists;
 
 public class PhoenixRelImplementorImpl implements PhoenixRelImplementor {
+    private final StatementContext statementContext;
     private final RuntimeContext runtimeContext;
 	private Stack<ImplementorContext> contextStack;
 	private SequenceManager sequenceManager;
 	private TableMapping tableMapping;
 	
-	public PhoenixRelImplementorImpl(RuntimeContext runtimeContext) {
+	public PhoenixRelImplementorImpl(
+	        StatementContext statementContext, RuntimeContext runtimeContext) {
+	    this.statementContext = statementContext;
 	    this.runtimeContext = runtimeContext;
 	    this.contextStack = new Stack<ImplementorContext>();
 	}
@@ -74,6 +78,11 @@ public class PhoenixRelImplementorImpl implements PhoenixRelImplementor {
             throw new RuntimeException(e);
         }
     }
+
+    @Override
+    public StatementContext getStatementContext() {
+        return statementContext;
+    }
     
     @Override
     public RuntimeContext getRuntimeContext() {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/75caf090/phoenix-core/src/main/java/org/apache/phoenix/calcite/rel/PhoenixTableScan.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/calcite/rel/PhoenixTableScan.java b/phoenix-core/src/main/java/org/apache/phoenix/calcite/rel/PhoenixTableScan.java
index c02b533..4474bac 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/calcite/rel/PhoenixTableScan.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/calcite/rel/PhoenixTableScan.java
@@ -128,7 +128,8 @@ public class PhoenixTableScan extends TableScan implements PhoenixQueryRel {
                 //    TODO instead of getting a random sample value, we'd better get it from
                 //    existing guidepost bytes.
                 // 3) test whether this dynamic filter is worth a recompile at runtime.
-                PhoenixRelImplementor tmpImplementor = new PhoenixRelImplementorImpl(RuntimeContext.EMPTY_CONTEXT) {                    
+                PhoenixRelImplementor tmpImplementor = new PhoenixRelImplementorImpl(
+                        context, RuntimeContext.EMPTY_CONTEXT) {                    
                     @SuppressWarnings("rawtypes")
                     @Override
                     public Expression newBindParameterExpression(int index, PDataType type, Integer maxLength) {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/75caf090/phoenix-core/src/main/java/org/apache/phoenix/calcite/rel/PhoenixToEnumerableConverter.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/calcite/rel/PhoenixToEnumerableConverter.java b/phoenix-core/src/main/java/org/apache/phoenix/calcite/rel/PhoenixToEnumerableConverter.java
index 5aef483..ad0f8c1 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/calcite/rel/PhoenixToEnumerableConverter.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/calcite/rel/PhoenixToEnumerableConverter.java
@@ -30,6 +30,7 @@ import org.apache.phoenix.calcite.rel.PhoenixRelImplementor.ImplementorContext;
 import org.apache.phoenix.compile.ExplainPlan;
 import org.apache.phoenix.compile.QueryPlan;
 import org.apache.phoenix.compile.RowProjector;
+import org.apache.phoenix.compile.StatementContext;
 import org.apache.phoenix.compile.StatementPlan;
 import org.apache.phoenix.execute.DelegateQueryPlan;
 import org.apache.phoenix.execute.RuntimeContext;
@@ -42,22 +43,26 @@ import org.apache.phoenix.iterate.ResultIterator;
  * Scan of a Phoenix table.
  */
 public class PhoenixToEnumerableConverter extends ConverterImpl implements EnumerableRel {
+    private final StatementContext context;
 
-    public static PhoenixToEnumerableConverter create(RelNode input) {
+    public static PhoenixToEnumerableConverter create(
+            RelNode input, StatementContext context) {
         RelOptCluster cluster = input.getCluster();
         RelTraitSet traits = input.getTraitSet().replace(EnumerableConvention.INSTANCE);
-        return new PhoenixToEnumerableConverter(cluster, traits, input);
+        return new PhoenixToEnumerableConverter(cluster, traits, input, context);
     }
 
     private PhoenixToEnumerableConverter(
         RelOptCluster cluster,
         RelTraitSet traits,
-        RelNode input) {
+        RelNode input,
+        StatementContext context) {
         super(cluster, ConventionTraitDef.INSTANCE, traits, input);
+        this.context = context;
     }
 
     @Override public RelNode copy(RelTraitSet traitSet, List<RelNode> inputs) {
-        return create(sole(inputs));
+        return create(sole(inputs), context);
     }
 
     @Override public RelOptCost computeSelfCost(RelOptPlanner planner, RelMetadataQuery mq) {
@@ -92,10 +97,11 @@ public class PhoenixToEnumerableConverter extends ConverterImpl implements Enume
         return implementor.result(physType, list.toBlock());
     }
     
-    static StatementPlan makePlan(PhoenixRel rel) {
+    StatementPlan makePlan(PhoenixRel rel) {
         RuntimeContext runtimeContext = new RuntimeContextImpl();
         RuntimeContext.THREAD_LOCAL.get().add(runtimeContext);
-        final PhoenixRelImplementor phoenixImplementor = new PhoenixRelImplementorImpl(runtimeContext);
+        final PhoenixRelImplementor phoenixImplementor =
+                new PhoenixRelImplementorImpl(context, runtimeContext);
         phoenixImplementor.pushContext(new ImplementorContext(true, false, ImmutableIntList.identity(rel.getRowType().getFieldCount())));
         final StatementPlan plan = rel.implement(phoenixImplementor);
         if (!(plan instanceof QueryPlan)) {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/75caf090/phoenix-core/src/main/java/org/apache/phoenix/calcite/rules/PhoenixConverterRules.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/calcite/rules/PhoenixConverterRules.java b/phoenix-core/src/main/java/org/apache/phoenix/calcite/rules/PhoenixConverterRules.java
index ca71ab1..ee444fd 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/calcite/rules/PhoenixConverterRules.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/calcite/rules/PhoenixConverterRules.java
@@ -59,6 +59,7 @@ import org.apache.phoenix.calcite.rel.PhoenixToEnumerableConverter;
 import org.apache.phoenix.calcite.rel.PhoenixUncollect;
 import org.apache.phoenix.calcite.rel.PhoenixUnion;
 import org.apache.phoenix.calcite.rel.PhoenixValues;
+import org.apache.phoenix.compile.StatementContext;
 
 import com.google.common.base.Predicate;
 import com.google.common.base.Predicates;
@@ -73,11 +74,6 @@ public class PhoenixConverterRules {
     private PhoenixConverterRules() {}
 
     public static final RelOptRule[] RULES = {
-        PhoenixToEnumerableConverterRule.SERVER,
-        PhoenixToEnumerableConverterRule.SERVERJOIN,
-        PhoenixToEnumerableConverterRule.SERVERAGG,
-        PhoenixToEnumerableConverterRule.CLIENT,
-        PhoenixToEnumerableConverterRule.MUTATION,
         PhoenixClientSortRule.INSTANCE,
         PhoenixServerSortRule.SERVER,
         PhoenixServerSortRule.SERVERJOIN,
@@ -101,11 +97,6 @@ public class PhoenixConverterRules {
     };
 
     public static final RelOptRule[] CONVERTIBLE_RULES = {
-        PhoenixToEnumerableConverterRule.SERVER,
-        PhoenixToEnumerableConverterRule.SERVERJOIN,
-        PhoenixToEnumerableConverterRule.SERVERAGG,
-        PhoenixToEnumerableConverterRule.CLIENT,
-        PhoenixToEnumerableConverterRule.MUTATION,
         PhoenixClientSortRule.INSTANCE,
         PhoenixServerSortRule.SERVER,
         PhoenixServerSortRule.SERVERJOIN,
@@ -872,24 +863,27 @@ public class PhoenixConverterRules {
      * {@link org.apache.calcite.adapter.enumerable.EnumerableConvention}.
      */
     public static class PhoenixToEnumerableConverterRule extends ConverterRule {
-        public static final ConverterRule SERVER =
-                new PhoenixToEnumerableConverterRule(PhoenixConvention.SERVER);
-        public static final ConverterRule SERVERJOIN =
-                new PhoenixToEnumerableConverterRule(PhoenixConvention.SERVERJOIN);
-        public static final ConverterRule SERVERAGG =
-                new PhoenixToEnumerableConverterRule(PhoenixConvention.SERVERAGG);
-        public static final ConverterRule CLIENT =
-                new PhoenixToEnumerableConverterRule(PhoenixConvention.CLIENT);
-        public static final ConverterRule MUTATION =
-                new PhoenixToEnumerableConverterRule(PhoenixConvention.MUTATION);
-
-        private PhoenixToEnumerableConverterRule(Convention inputConvention) {
+        private final StatementContext context;
+        public static ConverterRule[] createPhoenixToEnumerableConverterRules(
+                StatementContext context) {
+            return new ConverterRule[] {
+                new PhoenixToEnumerableConverterRule(PhoenixConvention.SERVER, context),
+                new PhoenixToEnumerableConverterRule(PhoenixConvention.SERVERJOIN, context),
+                new PhoenixToEnumerableConverterRule(PhoenixConvention.SERVERAGG, context),
+                new PhoenixToEnumerableConverterRule(PhoenixConvention.CLIENT, context),
+                new PhoenixToEnumerableConverterRule(PhoenixConvention.MUTATION, context)                    
+            };
+        }
+
+        private PhoenixToEnumerableConverterRule(
+                Convention inputConvention, StatementContext context) {
             super(RelNode.class, inputConvention, EnumerableConvention.INSTANCE,
                 "PhoenixToEnumerableConverterRule:" + inputConvention);
+            this.context = context;
         }
 
         @Override public RelNode convert(RelNode rel) {
-            return PhoenixToEnumerableConverter.create(rel);
+            return PhoenixToEnumerableConverter.create(rel, context);
         }
     }
     

http://git-wip-us.apache.org/repos/asf/phoenix/blob/75caf090/phoenix-core/src/test/java/org/apache/phoenix/calcite/ToExpressionTest.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/calcite/ToExpressionTest.java b/phoenix-core/src/test/java/org/apache/phoenix/calcite/ToExpressionTest.java
index ed1abd3..7dbca10 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/calcite/ToExpressionTest.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/calcite/ToExpressionTest.java
@@ -30,7 +30,6 @@ import org.apache.phoenix.compile.ExpressionCompiler;
 import org.apache.phoenix.compile.FromCompiler;
 import org.apache.phoenix.compile.StatementContext;
 import org.apache.phoenix.execute.RuntimeContext;
-import org.apache.phoenix.execute.RuntimeContextImpl;
 import org.apache.phoenix.expression.Expression;
 import org.apache.phoenix.jdbc.PhoenixConnection;
 import org.apache.phoenix.jdbc.PhoenixStatement;
@@ -62,11 +61,13 @@ public class ToExpressionTest extends BaseConnectionlessQueryTest {
 	    private final PTable table;
 	    private final Expression phoenixExpr;
 	    private final RexNode calciteExpr;
+	    private final StatementContext context;
 
 	    public ExpressionChecker(String tableName, String ddl, String sql, ExpressionGetter getter) throws Exception {
 	        Connection conn = DriverManager.getConnection(getOldUrl());
 	        PhoenixConnection pc = conn.unwrap(PhoenixConnection.class);
 	        PhoenixStatement stmt = conn.createStatement().unwrap(PhoenixStatement.class);
+	        this.context = new StatementContext(stmt);
 
 	        conn.createStatement().execute(ddl);
 	        this.table = pc.getMetaDataCache().getTableRef(new PTableKey(null,tableName)).getTable();
@@ -82,7 +83,8 @@ public class ToExpressionTest extends BaseConnectionlessQueryTest {
 	    }
 
 	    public ExpressionChecker checkExpressionEquality() {        
-	        PhoenixRelImplementor implementor = new PhoenixRelImplementorImpl(RuntimeContext.EMPTY_CONTEXT);
+	        PhoenixRelImplementor implementor =
+	                new PhoenixRelImplementorImpl(context, RuntimeContext.EMPTY_CONTEXT);
 	        implementor.setTableMapping(new TableMapping(table));
 	        Expression e = CalciteUtils.toExpression(this.calciteExpr, implementor);
 	        assertEquals(this.phoenixExpr,e);