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/03/17 22:46:24 UTC

phoenix git commit: Upgrade to calcite-1.1.0-incubating; Bug fixes in PhoenixRules

Repository: phoenix
Updated Branches:
  refs/heads/calcite 85937f7e9 -> 3d415c484


Upgrade to calcite-1.1.0-incubating; Bug fixes in PhoenixRules


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

Branch: refs/heads/calcite
Commit: 3d415c48402fb5575777c1bd31ede3325e60e877
Parents: 85937f7
Author: maryannxue <we...@intel.com>
Authored: Tue Mar 17 17:46:14 2015 -0400
Committer: maryannxue <we...@intel.com>
Committed: Tue Mar 17 17:46:14 2015 -0400

----------------------------------------------------------------------
 .../apache/phoenix/calcite/PhoenixRules.java    | 12 ++++-----
 .../apache/phoenix/calcite/PhoenixTable.java    | 27 ++++++++++++++++++--
 .../java/org/apache/phoenix/util/TestUtil.java  |  4 +--
 pom.xml                                         |  6 ++++-
 4 files changed, 38 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/3d415c48/phoenix-core/src/main/java/org/apache/phoenix/calcite/PhoenixRules.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/calcite/PhoenixRules.java b/phoenix-core/src/main/java/org/apache/phoenix/calcite/PhoenixRules.java
index b8551a6..e98ee73 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/calcite/PhoenixRules.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/calcite/PhoenixRules.java
@@ -68,7 +68,7 @@ public class PhoenixRules {
                 sort.getTraitSet().replace(out)
                     .replace(sort.getCollation());
             return new PhoenixSort(rel.getCluster(), traitSet,
-                convert(sort.getInput(), traitSet.replace(RelCollationImpl.EMPTY)),
+                convert(sort.getInput(), sort.getInput().getTraitSet().replace(RelCollationImpl.EMPTY)),
                 sort.getCollation(), sort.offset, sort.fetch);
         }
     }
@@ -91,7 +91,7 @@ public class PhoenixRules {
             return new PhoenixFilter(
                 rel.getCluster(),
                 traitSet,
-                convert(filter.getInput(), traitSet),
+                convert(filter.getInput(), filter.getInput().getTraitSet().replace(out)),
                 filter.getCondition());
         }
     }
@@ -112,7 +112,7 @@ public class PhoenixRules {
             final LogicalProject project = (LogicalProject) rel;
             final RelTraitSet traitSet = project.getTraitSet().replace(out);
             return new PhoenixProject(project.getCluster(), traitSet,
-                convert(project.getInput(), traitSet), project.getProjects(),
+                convert(project.getInput(), project.getInput().getTraitSet().replace(out)), project.getProjects(),
                 project.getRowType());
         }
     }
@@ -137,7 +137,7 @@ public class PhoenixRules {
                 return new PhoenixAggregate(
                     rel.getCluster(),
                     traitSet,
-                    convert(agg.getInput(), traitSet),
+                    convert(agg.getInput(), agg.getInput().getTraitSet().replace(out)),
                     agg.indicator,
                     agg.getGroupSet(),
                     agg.getGroupSets(),
@@ -186,8 +186,8 @@ public class PhoenixRules {
             final RelTraitSet traitSet =
                 join.getTraitSet().replace(out);
             return new PhoenixJoin(rel.getCluster(), traitSet,
-                convert(join.getLeft(), traitSet),
-                convert(join.getRight(), traitSet),
+                convert(join.getLeft(), join.getLeft().getTraitSet().replace(out)),
+                convert(join.getRight(), join.getRight().getTraitSet().replace(out)),
                 join.getCondition(),
                 join.getJoinType(),
                 join.getVariablesStopped());

http://git-wip-us.apache.org/repos/asf/phoenix/blob/3d415c48/phoenix-core/src/main/java/org/apache/phoenix/calcite/PhoenixTable.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/calcite/PhoenixTable.java b/phoenix-core/src/main/java/org/apache/phoenix/calcite/PhoenixTable.java
index 8a63aad..b2fba0a 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/calcite/PhoenixTable.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/calcite/PhoenixTable.java
@@ -1,8 +1,16 @@
 package org.apache.phoenix.calcite;
 
+import java.util.Collections;
+import java.util.List;
+
 import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+
 import org.apache.calcite.plan.RelOptCluster;
 import org.apache.calcite.plan.RelOptTable;
+import org.apache.calcite.rel.RelCollation;
+import org.apache.calcite.rel.RelDistribution;
+import org.apache.calcite.rel.RelDistributions;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rel.type.RelDataTypeFactory;
@@ -10,7 +18,6 @@ import org.apache.calcite.schema.Statistic;
 import org.apache.calcite.schema.TranslatableTable;
 import org.apache.calcite.schema.impl.AbstractTable;
 import org.apache.calcite.sql.type.SqlTypeName;
-
 import org.apache.calcite.util.ImmutableBitSet;
 import org.apache.phoenix.jdbc.PhoenixConnection;
 import org.apache.phoenix.schema.PColumn;
@@ -23,11 +30,17 @@ import org.apache.phoenix.schema.types.PDataType;
  */
 public class PhoenixTable extends AbstractTable implements TranslatableTable {
   public final PTable pTable;
+  public final ImmutableBitSet pkBitSet;
   public final PhoenixConnection pc;
 
   public PhoenixTable(PhoenixConnection pc, PTable pTable) {
       this.pc = Preconditions.checkNotNull(pc);
       this.pTable = Preconditions.checkNotNull(pTable);
+      List<Integer> pkPositions = Lists.<Integer> newArrayList();
+      for (PColumn column : pTable.getPKColumns()) {
+          pkPositions.add(column.getPosition());
+      }
+      this.pkBitSet = ImmutableBitSet.of(pkPositions);
     }
     
     public PTable getTable() {
@@ -70,7 +83,17 @@ public class PhoenixTable extends AbstractTable implements TranslatableTable {
 
             @Override
             public boolean isKey(ImmutableBitSet immutableBitSet) {
-                return false;
+                return immutableBitSet.contains(pkBitSet);
+            }
+
+            @Override
+            public List<RelCollation> getCollations() {
+                return Collections.<RelCollation> emptyList();
+            }
+
+            @Override
+            public RelDistribution getDistribution() {
+                return RelDistributions.RANDOM_DISTRIBUTED;
             }
         };
     }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/3d415c48/phoenix-core/src/test/java/org/apache/phoenix/util/TestUtil.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/util/TestUtil.java b/phoenix-core/src/test/java/org/apache/phoenix/util/TestUtil.java
index 541db14..220d465 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/util/TestUtil.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/util/TestUtil.java
@@ -201,8 +201,8 @@ public class TestUtil {
     public static final String JOIN_COITEM_TABLE_FULL_NAME = '"' + JOIN_SCHEMA + "\".\"" + JOIN_COITEM_TABLE + '"';
     public static final String JOIN_ORDER_TABLE_DISPLAY_NAME = JOIN_SCHEMA + "." + JOIN_ORDER_TABLE;
     public static final String JOIN_CUSTOMER_TABLE_DISPLAY_NAME = JOIN_SCHEMA + "." + JOIN_CUSTOMER_TABLE;
-    public static final String JOIN_ITEM_TABLE_DISPLAY_NAME = JOIN_SCHEMA + "." + JOIN_ITEM_TABLE;
-    public static final String JOIN_SUPPLIER_TABLE_DISPLAY_NAME = JOIN_SCHEMA + "." + JOIN_SUPPLIER_TABLE;
+    public static final String JOIN_ITEM_TABLE_DISPLAY_NAME = JOIN_ITEM_TABLE.toUpperCase();
+    public static final String JOIN_SUPPLIER_TABLE_DISPLAY_NAME = JOIN_SUPPLIER_TABLE.toUpperCase();
     public static final String JOIN_COITEM_TABLE_DISPLAY_NAME = JOIN_SCHEMA + "." + JOIN_COITEM_TABLE;
 
     /**

http://git-wip-us.apache.org/repos/asf/phoenix/blob/3d415c48/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 6f7454d..140e6bb 100644
--- a/pom.xml
+++ b/pom.xml
@@ -53,6 +53,10 @@
         <enabled>true</enabled>
       </snapshots>
     </repository>
+    <repository>
+      <id>apache calcite</id>
+      <url>https://repository.apache.org/content/repositories/orgapachecalcite-1006</url>
+    </repository>
   </repositories>
 
   <parent>
@@ -103,7 +107,7 @@
     <htrace.version>2.04</htrace.version>
     <collections.version>3.2.1</collections.version>
     <jodatime.version>2.3</jodatime.version>
-    <calcite.version>1.0.0-incubating-SNAPSHOT</calcite.version>
+    <calcite.version>1.1.0-incubating</calcite.version>
 
     <!-- Test Dependencies -->
     <mockito-all.version>1.8.5</mockito-all.version>