You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@vxquery.apache.org by ti...@apache.org on 2014/03/14 23:09:33 UTC

[12/25] git commit: Update to the rules and renaming of one.

Update to the rules and renaming of one.


Project: http://git-wip-us.apache.org/repos/asf/incubator-vxquery/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-vxquery/commit/6a4624e7
Tree: http://git-wip-us.apache.org/repos/asf/incubator-vxquery/tree/6a4624e7
Diff: http://git-wip-us.apache.org/repos/asf/incubator-vxquery/diff/6a4624e7

Branch: refs/heads/westmann/prettyprint
Commit: 6a4624e7b13c141fa8fd879f462da3d6e71ee83e
Parents: f72feda
Author: Preston Carman <pr...@apache.org>
Authored: Thu Feb 13 16:16:45 2014 -0800
Committer: Preston Carman <pr...@apache.org>
Committed: Thu Feb 13 16:16:45 2014 -0800

----------------------------------------------------------------------
 .../compiler/rewriter/RewriteRuleset.java       |  19 ++-
 .../rules/ConsolidateDataScanUnnestRule.java    | 151 -------------------
 .../rules/PushChildIntoDataScanRule.java        | 136 +++++++++++++++++
 3 files changed, 147 insertions(+), 159 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-vxquery/blob/6a4624e7/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/RewriteRuleset.java
----------------------------------------------------------------------
diff --git a/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/RewriteRuleset.java b/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/RewriteRuleset.java
index 4ffdd44..70a2b8e 100644
--- a/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/RewriteRuleset.java
+++ b/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/RewriteRuleset.java
@@ -20,7 +20,7 @@ import java.util.LinkedList;
 import java.util.List;
 
 import org.apache.vxquery.compiler.rewriter.rules.ConsolidateAssignAggregateRule;
-import org.apache.vxquery.compiler.rewriter.rules.ConsolidateDataScanUnnestRule;
+import org.apache.vxquery.compiler.rewriter.rules.PushChildIntoDataScanRule;
 import org.apache.vxquery.compiler.rewriter.rules.ConsolidateUnnestsRule;
 import org.apache.vxquery.compiler.rewriter.rules.ConvertAssignToAggregateRule;
 import org.apache.vxquery.compiler.rewriter.rules.ConvertAssignToUnnestRule;
@@ -30,6 +30,7 @@ import org.apache.vxquery.compiler.rewriter.rules.EliminateUnnestAggregateSubpla
 import org.apache.vxquery.compiler.rewriter.rules.IntroduceCollectionRule;
 import org.apache.vxquery.compiler.rewriter.rules.IntroduceTwoStepAggregateRule;
 import org.apache.vxquery.compiler.rewriter.rules.PushMapOperatorDownThroughProductRule;
+import org.apache.vxquery.compiler.rewriter.rules.RemoveRedundantCastExpressionsRule;
 import org.apache.vxquery.compiler.rewriter.rules.RemoveRedundantDataExpressionsRule;
 import org.apache.vxquery.compiler.rewriter.rules.RemoveRedundantPromoteExpressionsRule;
 import org.apache.vxquery.compiler.rewriter.rules.RemoveRedundantTreatExpressionsRule;
@@ -44,6 +45,7 @@ import edu.uci.ics.hyracks.algebricks.rewriter.rules.BreakSelectIntoConjunctsRul
 import edu.uci.ics.hyracks.algebricks.rewriter.rules.ComplexJoinInferenceRule;
 import edu.uci.ics.hyracks.algebricks.rewriter.rules.ConsolidateAssignsRule;
 import edu.uci.ics.hyracks.algebricks.rewriter.rules.ConsolidateSelectsRule;
+import edu.uci.ics.hyracks.algebricks.rewriter.rules.CopyLimitDownRule;
 import edu.uci.ics.hyracks.algebricks.rewriter.rules.EliminateSubplanRule;
 import edu.uci.ics.hyracks.algebricks.rewriter.rules.EnforceStructuralPropertiesRule;
 import edu.uci.ics.hyracks.algebricks.rewriter.rules.ExtractCommonOperatorsRule;
@@ -57,7 +59,6 @@ import edu.uci.ics.hyracks.algebricks.rewriter.rules.IntroduceGroupByCombinerRul
 import edu.uci.ics.hyracks.algebricks.rewriter.rules.IntroduceProjectsRule;
 import edu.uci.ics.hyracks.algebricks.rewriter.rules.IsolateHyracksOperatorsRule;
 import edu.uci.ics.hyracks.algebricks.rewriter.rules.PullSelectOutOfEqJoin;
-import edu.uci.ics.hyracks.algebricks.rewriter.rules.CopyLimitDownRule;
 import edu.uci.ics.hyracks.algebricks.rewriter.rules.PushProjectDownRule;
 import edu.uci.ics.hyracks.algebricks.rewriter.rules.PushProjectIntoDataSourceScanRule;
 import edu.uci.ics.hyracks.algebricks.rewriter.rules.PushSelectDownRule;
@@ -111,14 +112,11 @@ public class RewriteRuleset {
         normalization.add(new IntroduceCollectionRule());
         normalization.add(new RemoveUnusedAssignAndAggregateRule());
 
+        // Adds child steps to the data source scan.
         normalization.add(new ConsolidateUnnestsRule());
+        normalization.add(new PushChildIntoDataScanRule());
 
-        normalization.add(new RemoveRedundantTreatExpressionsRule());
-        normalization.add(new RemoveRedundantDataExpressionsRule());
-        normalization.add(new RemoveRedundantPromoteExpressionsRule());
-
-        normalization.add(new ConsolidateDataScanUnnestRule());
-
+        // Improvement for scalar child expressions
         normalization.add(new EliminateSubplanForSingleItemsRule());
         return normalization;
     }
@@ -158,6 +156,7 @@ public class RewriteRuleset {
         xquery.add(new InlineVariablesRule());
         xquery.add(new PushSelectDownRule());
         xquery.add(new PushSelectIntoJoinRule());
+        // Clean up
         xquery.add(new RemoveRedundantVariablesRule());
         xquery.add(new RemoveUnusedAssignAndAggregateRule());
         return xquery;
@@ -168,6 +167,10 @@ public class RewriteRuleset {
         normalization.add(new RemoveRedundantTreatExpressionsRule());
         normalization.add(new RemoveRedundantDataExpressionsRule());
         normalization.add(new RemoveRedundantPromoteExpressionsRule());
+        normalization.add(new RemoveRedundantCastExpressionsRule());
+        // Clean up
+        normalization.add(new RemoveRedundantVariablesRule());
+        normalization.add(new RemoveUnusedAssignAndAggregateRule());
         return normalization;
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-vxquery/blob/6a4624e7/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/rules/ConsolidateDataScanUnnestRule.java
----------------------------------------------------------------------
diff --git a/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/rules/ConsolidateDataScanUnnestRule.java b/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/rules/ConsolidateDataScanUnnestRule.java
deleted file mode 100644
index 8feb88b..0000000
--- a/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/rules/ConsolidateDataScanUnnestRule.java
+++ /dev/null
@@ -1,151 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.vxquery.compiler.rewriter.rules;
-
-import org.apache.commons.lang3.mutable.Mutable;
-import org.apache.commons.lang3.mutable.MutableObject;
-import org.apache.vxquery.compiler.rewriter.rules.util.ExpressionToolbox;
-import org.apache.vxquery.context.RootStaticContextImpl;
-import org.apache.vxquery.context.StaticContextImpl;
-import org.apache.vxquery.datamodel.accessors.TaggedValuePointable;
-import org.apache.vxquery.functions.BuiltinOperators;
-import org.apache.vxquery.functions.Function;
-import org.apache.vxquery.metadata.VXQueryCollectionDataSource;
-
-import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
-import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalExpression;
-import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalOperator;
-import edu.uci.ics.hyracks.algebricks.core.algebra.base.IOptimizationContext;
-import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalExpressionTag;
-import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
-import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
-import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.ConstantExpression;
-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AssignOperator;
-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.DataSourceScanOperator;
-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.UnnestOperator;
-import edu.uci.ics.hyracks.data.std.primitive.IntegerPointable;
-
-/**
- * The rule searches for an unnest operator immediately following a data scan
- * operator.
- * 
- * <pre>
- * Before 
- * 
- *   plan__parent
- *   UNNEST( $v2 : child( $v1 ) )
- *   DATASCAN( $source : $v1 )
- *   plan__child
- *   
- *   Where $v1 is not used in plan__parent.
- *   
- * After
- * 
- *   plan__parent
- *   DATASCAN( $source : $v1 )
- *   plan__child
- *   
- *   $source is encoded with the child parameters.
- * </pre>
- * 
- * @author prestonc
- */
-public class ConsolidateDataScanUnnestRule extends AbstractUsedVariablesProcessingRule {
-    final StaticContextImpl dCtx = new StaticContextImpl(RootStaticContextImpl.INSTANCE);
-    final int ARG_DATA = 0;
-    final int ARG_TYPE = 1;
-
-    protected boolean processOperator(Mutable<ILogicalOperator> opRef, IOptimizationContext context)
-            throws AlgebricksException {
-        AbstractLogicalOperator op0 = (AbstractLogicalOperator) opRef.getValue();
-        if (op0.getInputs().isEmpty()) {
-            return false;
-        }
-
-        AbstractLogicalOperator op1 = (AbstractLogicalOperator) op0.getInputs().get(0).getValue();
-        if (op1.getOperatorTag() != LogicalOperatorTag.UNNEST) {
-            return false;
-        }
-        UnnestOperator unnest = (UnnestOperator) op1;
-
-        AbstractLogicalOperator op2 = (AbstractLogicalOperator) unnest.getInputs().get(0).getValue();
-        if (op2.getOperatorTag() != LogicalOperatorTag.DATASOURCESCAN) {
-            return false;
-        }
-        DataSourceScanOperator datascan = (DataSourceScanOperator) op2;
-
-        if (!usedVariables.contains(datascan.getVariables())) {
-            // Check to see if the unnest expression is a child function.
-            ILogicalExpression logicalExpression = (ILogicalExpression) unnest.getExpressionRef().getValue();
-            if (logicalExpression.getExpressionTag() != LogicalExpressionTag.FUNCTION_CALL) {
-                return false;
-            }
-            AbstractFunctionCallExpression functionCall = (AbstractFunctionCallExpression) logicalExpression;
-            Function functionInfo = (Function) functionCall.getFunctionInfo();
-            if (!functionInfo.getFunctionIdentifier().equals(BuiltinOperators.CHILD.getFunctionIdentifier())) {
-                return false;
-            }
-
-            // Find all child functions.
-            VXQueryCollectionDataSource ds = (VXQueryCollectionDataSource) datascan.getDataSource();
-            updateDataSource(ds, unnest.getExpressionRef());
-
-            // Replace unnest with noop assign. Keeps variable chain.
-            Mutable<ILogicalExpression> varExp = ExpressionToolbox.findVariableExpression(unnest.getExpressionRef(),
-                    datascan.getVariables().get(0));
-            AssignOperator noOp = new AssignOperator(unnest.getVariable(), varExp);
-            noOp.getInputs().addAll(unnest.getInputs());
-            op0.getInputs().clear();
-            op0.getInputs().add(new MutableObject<ILogicalOperator>(noOp));
-            return true;
-        }
-        return false;
-    }
-
-    /**
-     * In reverse add them to the data source.
-     * 
-     * @param ds
-     * @param expression
-     */
-    private void updateDataSource(VXQueryCollectionDataSource ds, Mutable<ILogicalExpression> expression) {
-        ILogicalExpression logicalExpression = (ILogicalExpression) expression.getValue();
-        if (logicalExpression.getExpressionTag() != LogicalExpressionTag.FUNCTION_CALL) {
-            return;
-        }
-        AbstractFunctionCallExpression functionCall = (AbstractFunctionCallExpression) logicalExpression;
-        Function functionInfo = (Function) functionCall.getFunctionInfo();
-        if (!functionInfo.getFunctionIdentifier().equals(BuiltinOperators.CHILD.getFunctionIdentifier())) {
-            return;
-        }
-        // Traverse down child function nesting.
-        updateDataSource(ds, functionCall.getArguments().get(ARG_DATA));
-
-        // Add the child type parameter to data source.
-        ILogicalExpression argType = functionCall.getArguments().get(ARG_TYPE).getValue();
-        if (argType.getExpressionTag() != LogicalExpressionTag.CONSTANT) {
-            return;
-        }
-        TaggedValuePointable tvp = new TaggedValuePointable();
-        ExpressionToolbox.getConstantAsPointable((ConstantExpression) argType, tvp);
-
-        IntegerPointable pTypeCode = (IntegerPointable) IntegerPointable.FACTORY.createPointable();
-        tvp.getValue(pTypeCode);
-        ds.addChildSeq(pTypeCode.getInteger());
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-vxquery/blob/6a4624e7/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/rules/PushChildIntoDataScanRule.java
----------------------------------------------------------------------
diff --git a/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/rules/PushChildIntoDataScanRule.java b/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/rules/PushChildIntoDataScanRule.java
new file mode 100644
index 0000000..d49d696
--- /dev/null
+++ b/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/rules/PushChildIntoDataScanRule.java
@@ -0,0 +1,136 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.vxquery.compiler.rewriter.rules;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.ListIterator;
+
+import org.apache.commons.lang3.mutable.Mutable;
+import org.apache.commons.lang3.mutable.MutableObject;
+import org.apache.vxquery.compiler.rewriter.rules.util.ExpressionToolbox;
+import org.apache.vxquery.context.RootStaticContextImpl;
+import org.apache.vxquery.context.StaticContextImpl;
+import org.apache.vxquery.datamodel.accessors.TaggedValuePointable;
+import org.apache.vxquery.functions.BuiltinFunctions;
+import org.apache.vxquery.functions.BuiltinOperators;
+import org.apache.vxquery.functions.Function;
+import org.apache.vxquery.metadata.VXQueryCollectionDataSource;
+import org.apache.vxquery.types.SequenceType;
+
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalExpressionTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.ConstantExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AssignOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.DataSourceScanOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.UnnestOperator;
+import edu.uci.ics.hyracks.data.std.primitive.IntegerPointable;
+
+/**
+ * The rule searches for an unnest operator immediately following a data scan
+ * operator.
+ * 
+ * <pre>
+ * Before 
+ * 
+ *   plan__parent
+ *   UNNEST( $v2 : child( $v1 ) )
+ *   DATASCAN( $source : $v1 )
+ *   plan__child
+ *   
+ *   Where $v1 is not used in plan__parent.
+ *   
+ * After
+ * 
+ *   plan__parent
+ *   DATASCAN( $source : $v1 )
+ *   plan__child
+ *   
+ *   $source is encoded with the child parameters.
+ * </pre>
+ * 
+ * @author prestonc
+ */
+public class PushChildIntoDataScanRule extends AbstractUsedVariablesProcessingRule {
+    final StaticContextImpl dCtx = new StaticContextImpl(RootStaticContextImpl.INSTANCE);
+    final int ARG_DATA = 0;
+    final int ARG_TYPE = 1;
+
+    protected boolean processOperator(Mutable<ILogicalOperator> opRef, IOptimizationContext context)
+            throws AlgebricksException {
+        AbstractLogicalOperator op0 = (AbstractLogicalOperator) opRef.getValue();
+        if (op0.getInputs().isEmpty()) {
+            return false;
+        }
+
+        AbstractLogicalOperator op1 = (AbstractLogicalOperator) op0.getInputs().get(0).getValue();
+        if (op1.getOperatorTag() != LogicalOperatorTag.UNNEST) {
+            return false;
+        }
+        UnnestOperator unnest = (UnnestOperator) op1;
+
+        AbstractLogicalOperator op2 = (AbstractLogicalOperator) unnest.getInputs().get(0).getValue();
+        if (op2.getOperatorTag() != LogicalOperatorTag.DATASOURCESCAN) {
+            return false;
+        }
+        DataSourceScanOperator datascan = (DataSourceScanOperator) op2;
+
+        if (!usedVariables.contains(datascan.getVariables())) {
+            // Find all child functions.
+            VXQueryCollectionDataSource ds = (VXQueryCollectionDataSource) datascan.getDataSource();
+            if (!updateDataSource(ds, unnest.getExpressionRef())) {
+                return false;
+            }
+
+            // Replace unnest with noop assign. Keeps variable chain.
+            Mutable<ILogicalExpression> varExp = ExpressionToolbox.findVariableExpression(unnest.getExpressionRef(),
+                    datascan.getVariables().get(0));
+            AssignOperator noOp = new AssignOperator(unnest.getVariable(), varExp);
+            noOp.getInputs().addAll(unnest.getInputs());
+            op0.getInputs().clear();
+            op0.getInputs().add(new MutableObject<ILogicalOperator>(noOp));
+            return true;
+        }
+        return false;
+    }
+
+    /**
+     * In reverse add them to the data source.
+     * 
+     * @param ds
+     * @param expression
+     */
+    private boolean updateDataSource(VXQueryCollectionDataSource ds, Mutable<ILogicalExpression> expression) {
+        boolean added = false;
+        List<Mutable<ILogicalExpression>> finds = new ArrayList<Mutable<ILogicalExpression>>();
+        ExpressionToolbox.findAllFunctionExpressions(expression, BuiltinOperators.CHILD.getFunctionIdentifier(), finds);
+        for (int i = finds.size(); i > 0; --i) {
+            int typeId = ExpressionToolbox.getTypeExpressionTypeArgument(finds.get(i - 1));
+            if (typeId > 0) {
+                ds.addChildSeq(typeId);
+                added = true;
+            }
+        }
+        return added;
+    }
+}