You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@vxquery.apache.org by pr...@apache.org on 2013/10/24 23:42:36 UTC

svn commit: r1535567 - in /incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/rules: RemoveUnusedSortDistinctNodesRule.java util/CardinalityRuleToolbox.java

Author: prestonc
Date: Thu Oct 24 21:42:36 2013
New Revision: 1535567

URL: http://svn.apache.org/r1535567
Log:
Updated these rules to support the new join operators that are being added to the query plans.

Modified:
    incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/rules/RemoveUnusedSortDistinctNodesRule.java
    incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/rules/util/CardinalityRuleToolbox.java

Modified: incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/rules/RemoveUnusedSortDistinctNodesRule.java
URL: http://svn.apache.org/viewvc/incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/rules/RemoveUnusedSortDistinctNodesRule.java?rev=1535567&r1=1535566&r2=1535567&view=diff
==============================================================================
--- incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/rules/RemoveUnusedSortDistinctNodesRule.java (original)
+++ incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/rules/RemoveUnusedSortDistinctNodesRule.java Thu Oct 24 21:42:36 2013
@@ -42,6 +42,8 @@ import edu.uci.ics.hyracks.algebricks.co
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractOperatorWithNestedPlans;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AggregateOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AssignOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.InnerJoinOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.LeftOuterJoinOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.NestedTupleSourceOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.OrderOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.SubplanOperator;
@@ -336,45 +338,60 @@ public class RemoveUnusedSortDistinctNod
         switch (op.getOperatorTag()) {
             case AGGREGATE:
                 AggregateOperator aggregate = (AggregateOperator) op;
-                ILogicalExpression aggregateLogicalExpression = (ILogicalExpression) aggregate.getExpressions().get(0)
-                        .getValue();
-                variableId = aggregate.getVariables().get(0).getId();
-                documentOrder = propagateDocumentOrder(aggregateLogicalExpression, documentOrderVariablesForOperator);
-                uniqueNodes = propagateUniqueNodes(aggregateLogicalExpression, uniqueNodesVariablesForOperator);
-                documentOrderVariables.put(variableId, documentOrder);
-                uniqueNodesVariables.put(variableId, uniqueNodes);
+                for (int index = 0; index < aggregate.getExpressions().size(); index++) {
+                    ILogicalExpression aggregateLogicalExpression = (ILogicalExpression) aggregate.getExpressions()
+                            .get(index).getValue();
+                    variableId = aggregate.getVariables().get(index).getId();
+                    documentOrder = propagateDocumentOrder(aggregateLogicalExpression,
+                            documentOrderVariablesForOperator);
+                    uniqueNodes = propagateUniqueNodes(aggregateLogicalExpression, uniqueNodesVariablesForOperator);
+                    documentOrderVariables.put(variableId, documentOrder);
+                    uniqueNodesVariables.put(variableId, uniqueNodes);
+                }
                 break;
             case ASSIGN:
                 AssignOperator assign = (AssignOperator) op;
-                ILogicalExpression assignLogicalExpression = (ILogicalExpression) assign.getExpressions().get(0)
-                        .getValue();
-                variableId = assign.getVariables().get(0).getId();
-                documentOrder = propagateDocumentOrder(assignLogicalExpression, documentOrderVariablesForOperator);
-                uniqueNodes = propagateUniqueNodes(assignLogicalExpression, uniqueNodesVariablesForOperator);
-                documentOrderVariables.put(variableId, documentOrder);
-                uniqueNodesVariables.put(variableId, uniqueNodes);
+                for (int index = 0; index < assign.getExpressions().size(); index++) {
+                    ILogicalExpression assignLogicalExpression = (ILogicalExpression) assign.getExpressions()
+                            .get(index).getValue();
+                    variableId = assign.getVariables().get(index).getId();
+                    documentOrder = propagateDocumentOrder(assignLogicalExpression, documentOrderVariablesForOperator);
+                    uniqueNodes = propagateUniqueNodes(assignLogicalExpression, uniqueNodesVariablesForOperator);
+                    documentOrderVariables.put(variableId, documentOrder);
+                    uniqueNodesVariables.put(variableId, uniqueNodes);
+                }
+                break;
+            case INNERJOIN:
+            case LEFTOUTERJOIN:
+                resetDocumentOrderVariables(documentOrderVariables, DocumentOrder.NO);
+                resetUniqueNodesVariables(uniqueNodesVariables, UniqueNodes.NO);
                 break;
             case ORDER:
                 // Get order variable id that is altered.
                 OrderOperator order = (OrderOperator) op;
-                ILogicalExpression orderLogicalExpression = order.getOrderExpressions().get(0).second.getValue();
-                if (orderLogicalExpression.getExpressionTag() != LogicalExpressionTag.VARIABLE) {
-                    throw new RuntimeException("Operator (" + op.getOperatorTag()
-                            + ") has received unexpected input in rewrite rule.");
-                }
-                VariableReferenceExpression variableExpression = (VariableReferenceExpression) orderLogicalExpression;
-                variableId = variableExpression.getVariableReference().getId();
+                for (int index = 0; index < order.getOrderExpressions().size(); index++) {
+                    ILogicalExpression orderLogicalExpression = order.getOrderExpressions().get(index).second
+                            .getValue();
+                    if (orderLogicalExpression.getExpressionTag() != LogicalExpressionTag.VARIABLE) {
+                        throw new RuntimeException("Operator (" + op.getOperatorTag()
+                                + ") has received unexpected input in rewrite rule.");
+                    }
+                    VariableReferenceExpression variableExpression = (VariableReferenceExpression) orderLogicalExpression;
+                    variableId = variableExpression.getVariableReference().getId();
 
-                // Remove document order from variable used in order operator.
-                documentOrderVariables.put(variableId, DocumentOrder.NO);
+                    // Remove document order from variable used in order operator.
+                    documentOrderVariables.put(variableId, DocumentOrder.NO);
+                }
                 break;
             case SUBPLAN:
                 // Find the last operator to set a variable and call this function again.
                 SubplanOperator subplan = (SubplanOperator) op;
-                AbstractLogicalOperator lastOperator = (AbstractLogicalOperator) subplan.getNestedPlans().get(0)
-                        .getRoots().get(0).getValue();
-                updateVariableMap(lastOperator, cardinalityVariable, documentOrderVariables, uniqueNodesVariables,
-                        vxqueryContext);
+                for (int index = 0; index < subplan.getNestedPlans().size(); index++) {
+                    AbstractLogicalOperator lastOperator = (AbstractLogicalOperator) subplan.getNestedPlans().get(index)
+                            .getRoots().get(0).getValue();
+                    updateVariableMap(lastOperator, cardinalityVariable, documentOrderVariables, uniqueNodesVariables,
+                            vxqueryContext);
+                }
                 break;
             case UNNEST:
                 // Get unnest item property.
@@ -427,9 +444,7 @@ public class RemoveUnusedSortDistinctNod
             case EXTENSION_OPERATOR:
             case GROUP:
             case INDEX_INSERT_DELETE:
-            case INNERJOIN:
             case INSERT_DELETE:
-            case LEFTOUTERJOIN:
             case LIMIT:
             case PARTITIONINGSPLIT:
             case PROJECT:

Modified: incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/rules/util/CardinalityRuleToolbox.java
URL: http://svn.apache.org/viewvc/incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/rules/util/CardinalityRuleToolbox.java?rev=1535567&r1=1535566&r2=1535567&view=diff
==============================================================================
--- incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/rules/util/CardinalityRuleToolbox.java (original)
+++ incubator/vxquery/trunk/vxquery/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/rules/util/CardinalityRuleToolbox.java Thu Oct 24 21:42:36 2013
@@ -60,6 +60,8 @@ public class CardinalityRuleToolbox {
                 cardinalityVariable = vxqueryContext.getCardinalityOperatorMap(lastOperator);
                 break;
             case DATASOURCESCAN:
+            case INNERJOIN:
+            case LEFTOUTERJOIN:
             case UNNEST:
                 cardinalityVariable = Cardinality.MANY;
                 break;
@@ -83,9 +85,7 @@ public class CardinalityRuleToolbox {
             case DISTINCT:
             case EXTENSION_OPERATOR:
             case INDEX_INSERT_DELETE:
-            case INNERJOIN:
             case INSERT_DELETE:
-            case LEFTOUTERJOIN:
             case PARTITIONINGSPLIT:
             case REPLICATE:
             case RUNNINGAGGREGATE: