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/10/03 22:16:23 UTC

phoenix git commit: Avoid adding redundant hooks; set materialization_enabled=true by default

Repository: phoenix
Updated Branches:
  refs/heads/calcite 14c217bf8 -> 3889209d1


Avoid adding redundant hooks; set materialization_enabled=true by default


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

Branch: refs/heads/calcite
Commit: 3889209d13f2154b96ee7c5b4e0017c324790899
Parents: 14c217b
Author: maryannxue <ma...@gmail.com>
Authored: Mon Oct 3 15:16:16 2016 -0700
Committer: maryannxue <ma...@gmail.com>
Committed: Mon Oct 3 15:16:16 2016 -0700

----------------------------------------------------------------------
 .../calcite/jdbc/PhoenixCalciteFactory.java     | 56 ++++++++++++++++++--
 .../jdbc/PhoenixCalciteEmbeddedDriver.java      |  4 ++
 .../calcite/jdbc/PhoenixPrepareImpl.java        | 39 --------------
 3 files changed, 55 insertions(+), 44 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/3889209d/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 014891b..6b00f04 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
@@ -35,14 +35,24 @@ import org.apache.calcite.jdbc.CalciteFactory;
 import org.apache.calcite.jdbc.Driver;
 import org.apache.calcite.linq4j.Enumerable;
 import org.apache.calcite.linq4j.Ord;
+import org.apache.calcite.prepare.Prepare.Materialization;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.runtime.Hook;
+import org.apache.calcite.runtime.Hook.Closeable;
 import org.apache.calcite.schema.SchemaPlus;
+import org.apache.calcite.tools.Program;
+import org.apache.calcite.tools.Programs;
+import org.apache.calcite.util.Holder;
 import org.apache.phoenix.calcite.PhoenixSchema;
+import org.apache.phoenix.calcite.rel.PhoenixRel;
 import org.apache.phoenix.exception.SQLExceptionCode;
 import org.apache.phoenix.exception.SQLExceptionInfo;
 import org.apache.phoenix.execute.RuntimeContext;
 import org.apache.phoenix.jdbc.PhoenixConnection;
 
+import com.google.common.base.Function;
 import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
 
 public class PhoenixCalciteFactory extends CalciteFactory {
@@ -59,7 +69,8 @@ public class PhoenixCalciteFactory extends CalciteFactory {
         AvaticaFactory factory, String url, Properties info,
         CalciteSchema rootSchema, JavaTypeFactory typeFactory) {
         return new PhoenixCalciteConnection(
-                (Driver) driver, factory, url, info, rootSchema, typeFactory);
+                (Driver) driver, factory, url, info,
+                CalciteSchema.createRootSchema(true, false), typeFactory);
     }
 
     @Override
@@ -108,14 +119,46 @@ public class PhoenixCalciteFactory extends CalciteFactory {
     }
 
     private static class PhoenixCalciteConnection extends CalciteConnectionImpl {
-        final Map<Meta.StatementHandle, ImmutableList<RuntimeContext>> runtimeContextMap =
+        private final Map<Meta.StatementHandle, ImmutableList<RuntimeContext>> runtimeContextMap =
                 new ConcurrentHashMap<Meta.StatementHandle, ImmutableList<RuntimeContext>>();
+        private final List<Closeable> hooks = Lists.newArrayList();
         
         public PhoenixCalciteConnection(Driver driver, AvaticaFactory factory, String url,
-                Properties info, CalciteSchema rootSchema,
+                Properties info, final CalciteSchema rootSchema,
                 JavaTypeFactory typeFactory) {
-            super(driver, factory, url, info,
-                    CalciteSchema.createRootSchema(true, false), typeFactory);
+            super(driver, factory, url, info, rootSchema, typeFactory);
+
+            hooks.add(Hook.PARSE_TREE.add(new Function<Object[], Object>() {
+                @Override
+                public Object apply(Object[] input) {
+                    // TODO Auto-generated method stub
+                    return null;
+                }            
+            }));
+
+            hooks.add(Hook.TRIMMED.add(new Function<RelNode, Object>() {
+                @Override
+                public Object apply(RelNode root) {
+                    for (CalciteSchema schema : rootSchema.getSubSchemaMap().values()) {
+                        if (schema.schema instanceof PhoenixSchema) {
+                            ((PhoenixSchema) schema.schema).defineIndexesAsMaterializations();
+                            for (CalciteSchema subSchema : schema.getSubSchemaMap().values()) {
+                                ((PhoenixSchema) subSchema.schema).defineIndexesAsMaterializations();
+                            }
+                        }
+                    }
+                    return null;
+                }
+            }));
+
+            hooks.add(Hook.PROGRAM.add(new Function<org.apache.calcite.util.Pair<List<Materialization>, Holder<Program>>, Object>() {
+                @Override
+                public Object apply(
+                        org.apache.calcite.util.Pair<List<Materialization>, Holder<Program>> input) {
+                    input.getValue().set(Programs.standard(PhoenixRel.METADATA_PROVIDER));
+                    return null;
+                }
+            }));
         }
 
         public <T> Enumerable<T> enumerable(Meta.StatementHandle handle,
@@ -230,6 +273,9 @@ public class PhoenixCalciteFactory extends CalciteFactory {
                 public void call(PhoenixConnection conn) throws SQLException {
                     conn.close();
                 }});
+            for (Closeable hook : hooks) {
+                hook.close();
+            }
             super.close();
         }
         

http://git-wip-us.apache.org/repos/asf/phoenix/blob/3889209d/phoenix-core/src/main/java/org/apache/phoenix/calcite/jdbc/PhoenixCalciteEmbeddedDriver.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/calcite/jdbc/PhoenixCalciteEmbeddedDriver.java b/phoenix-core/src/main/java/org/apache/phoenix/calcite/jdbc/PhoenixCalciteEmbeddedDriver.java
index c370e84..be2ef36 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/calcite/jdbc/PhoenixCalciteEmbeddedDriver.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/calcite/jdbc/PhoenixCalciteEmbeddedDriver.java
@@ -93,6 +93,10 @@ public abstract class PhoenixCalciteEmbeddedDriver extends Driver implements SQL
                 info2,
                 CalciteConnectionProperty.CONFORMANCE.camelName(),
                 SqlConformance.ORACLE_10.toString());
+        setPropertyIfNotSpecified(
+                info2,
+                CalciteConnectionProperty.MATERIALIZATIONS_ENABLED.camelName(),
+                Boolean.TRUE.toString());
         
         final String prefix = getConnectStringPrefix();
         assert url.startsWith(prefix);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/3889209d/phoenix-core/src/main/java/org/apache/phoenix/calcite/jdbc/PhoenixPrepareImpl.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/calcite/jdbc/PhoenixPrepareImpl.java b/phoenix-core/src/main/java/org/apache/phoenix/calcite/jdbc/PhoenixPrepareImpl.java
index f7e4a45..ce5f865 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/calcite/jdbc/PhoenixPrepareImpl.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/calcite/jdbc/PhoenixPrepareImpl.java
@@ -7,20 +7,16 @@ import java.util.Map;
 
 import org.apache.calcite.adapter.enumerable.EnumerableRules;
 import org.apache.calcite.jdbc.CalcitePrepare;
-import org.apache.calcite.jdbc.CalciteSchema;
 import org.apache.calcite.plan.RelOptCluster;
 import org.apache.calcite.plan.RelOptCostFactory;
 import org.apache.calcite.plan.RelOptPlanner;
 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.logical.LogicalSort;
 import org.apache.calcite.rel.rules.JoinCommuteRule;
 import org.apache.calcite.rel.rules.SortProjectTransposeRule;
 import org.apache.calcite.rel.rules.SortUnionTransposeRule;
 import org.apache.calcite.rex.RexBuilder;
-import org.apache.calcite.runtime.Hook;
 import org.apache.calcite.schema.SchemaPlus;
 import org.apache.calcite.sql.SqlColumnDefInPkConstraintNode;
 import org.apache.calcite.sql.SqlColumnDefNode;
@@ -34,9 +30,6 @@ import org.apache.calcite.sql.SqlOptionNode;
 import org.apache.calcite.sql.parser.SqlParser;
 import org.apache.calcite.sql.parser.SqlParserPos;
 import org.apache.calcite.sql.parser.SqlParserUtil;
-import org.apache.calcite.tools.Program;
-import org.apache.calcite.tools.Programs;
-import org.apache.calcite.util.Holder;
 import org.apache.calcite.util.NlsString;
 import org.apache.hadoop.hbase.util.Pair;
 import org.apache.phoenix.calcite.PhoenixSchema;
@@ -90,7 +83,6 @@ import org.apache.phoenix.schema.PTable.IndexType;
 import org.apache.phoenix.schema.PTableType;
 import org.apache.phoenix.schema.SortOrder;
 
-import com.google.common.base.Function;
 import com.google.common.collect.ArrayListMultimap;
 import com.google.common.collect.ListMultimap;
 import com.google.common.collect.Lists;
@@ -156,37 +148,6 @@ public class PhoenixPrepareImpl extends CalcitePrepareImpl {
         planner.addRule(new PhoenixForwardTableScanRule(PhoenixTemporarySort.class));
         planner.addRule(new PhoenixReverseTableScanRule(LogicalSort.class));
         planner.addRule(new PhoenixReverseTableScanRule(PhoenixTemporarySort.class));
-                
-        if (prepareContext.config().materializationsEnabled()) {
-            final CalciteSchema rootSchema = prepareContext.getRootSchema();
-            Hook.TRIMMED.add(new Function<RelNode, Object>() {
-                boolean called = false;
-                @Override
-                public Object apply(RelNode root) {
-                    if (!called) {
-                        called = true;
-                        for (CalciteSchema schema : rootSchema.getSubSchemaMap().values()) {
-                            if (schema.schema instanceof PhoenixSchema) {
-                                ((PhoenixSchema) schema.schema).defineIndexesAsMaterializations();
-                                for (CalciteSchema subSchema : schema.getSubSchemaMap().values()) {
-                                    ((PhoenixSchema) subSchema.schema).defineIndexesAsMaterializations();
-                                }
-                            }
-                        }
-                    }
-                    return null;
-                }            
-            });
-        }
-        
-        Hook.PROGRAM.add(new Function<org.apache.calcite.util.Pair<List<Materialization>, Holder<Program>>, Object>() {
-			@Override
-			public Object apply(
-			        org.apache.calcite.util.Pair<List<Materialization>, Holder<Program>> input) {
-				input.getValue().set(Programs.standard(PhoenixRel.METADATA_PROVIDER));
-				return null;
-			}
-        });
 
         return planner;
     }