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:29 UTC

[08/25] git commit: All the redundant type expression rules have been updated to improve code reuse and new use cases.

All the redundant type expression rules have been updated to improve code reuse and new use cases.


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

Branch: refs/heads/westmann/prettyprint
Commit: c74c98656f9afd9ea7ead84486511c6964ff78a2
Parents: 6d7c53e
Author: Preston Carman <pr...@apache.org>
Authored: Thu Feb 13 15:06:38 2014 -0800
Committer: Preston Carman <pr...@apache.org>
Committed: Thu Feb 13 15:06:38 2014 -0800

----------------------------------------------------------------------
 ...tractRemoveRedundantTypeExpressionsRule.java | 97 ++++++++++++++++++++
 .../RemoveRedundantCastExpressionsRule.java     | 61 ++++++++++++
 .../RemoveRedundantDataExpressionsRule.java     | 67 ++------------
 .../RemoveRedundantPromoteExpressionsRule.java  | 92 ++++---------------
 .../RemoveRedundantTreatExpressionsRule.java    | 87 +++---------------
 .../rewriter/rules/util/ExpressionToolbox.java  | 92 ++++++++++++++++++-
 .../rewriter/rules/util/OperatorToolbox.java    | 85 ++++++++++++++---
 .../functions/type/SequenceTypeMatcher.java     | 32 ++++++-
 8 files changed, 394 insertions(+), 219 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-vxquery/blob/c74c9865/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/rules/AbstractRemoveRedundantTypeExpressionsRule.java
----------------------------------------------------------------------
diff --git a/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/rules/AbstractRemoveRedundantTypeExpressionsRule.java b/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/rules/AbstractRemoveRedundantTypeExpressionsRule.java
new file mode 100644
index 0000000..752e107
--- /dev/null
+++ b/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/rules/AbstractRemoveRedundantTypeExpressionsRule.java
@@ -0,0 +1,97 @@
+/*
+ * 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 org.apache.commons.lang3.mutable.Mutable;
+import org.apache.vxquery.compiler.rewriter.rules.util.ExpressionToolbox;
+import org.apache.vxquery.compiler.rewriter.rules.util.OperatorToolbox;
+import org.apache.vxquery.context.RootStaticContextImpl;
+import org.apache.vxquery.context.StaticContextImpl;
+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.expressions.AbstractFunctionCallExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import edu.uci.ics.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+
+public abstract class AbstractRemoveRedundantTypeExpressionsRule implements IAlgebraicRewriteRule {
+    final StaticContextImpl dCtx = new StaticContextImpl(RootStaticContextImpl.INSTANCE);
+    final int ARG_DATA = 0;
+    final int ARG_TYPE = 1;
+    final List<Mutable<ILogicalExpression>> functionList = new ArrayList<Mutable<ILogicalExpression>>();
+    
+    protected abstract FunctionIdentifier getSearchFunction();
+
+    @Override
+    public boolean rewritePre(Mutable<ILogicalOperator> opRef, IOptimizationContext context) throws AlgebricksException {
+        return false;
+    }
+
+    @Override
+    public boolean rewritePost(Mutable<ILogicalOperator> opRef, IOptimizationContext context)
+            throws AlgebricksException {
+        boolean modified = false;
+        List<Mutable<ILogicalExpression>> expressions = OperatorToolbox.getExpressions(opRef);
+        for (Mutable<ILogicalExpression> expression : expressions) {
+            if (processTypeExpression(opRef, expression)) {
+                modified = true;
+            }
+        }
+        return modified;
+    }
+
+    private boolean processTypeExpression(Mutable<ILogicalOperator> opRef, Mutable<ILogicalExpression> search) {
+        boolean modified = false;
+        SequenceType inputSequenceType;
+        functionList.clear();
+        ExpressionToolbox.findAllFunctionExpressions(search, getSearchFunction(), functionList);
+        for (Mutable<ILogicalExpression> searchM : functionList) {
+            // Get input function
+            AbstractFunctionCallExpression searchFunction = (AbstractFunctionCallExpression) searchM.getValue();
+            Mutable<ILogicalExpression> argFirstM = searchFunction.getArguments().get(ARG_DATA);
+
+            // Find the input return type.
+            inputSequenceType = ExpressionToolbox.getOutputSequenceType(opRef, argFirstM, dCtx);
+
+            // Find the treat type.
+            SequenceType sTypeArg = null;
+            if (hasTypeArgument()) {
+                sTypeArg = ExpressionToolbox.getTypeExpressionTypeArguement(searchM, dCtx);
+            }
+
+            // remove
+            if (safeToReplace(sTypeArg, inputSequenceType)) {
+                searchM.setValue(argFirstM.getValue());
+                modified = true;
+            }
+        }
+        return modified;
+    }
+
+    public abstract boolean safeToReplace(SequenceType sTypeArg, SequenceType sTypeOutput);
+
+    public boolean hasTypeArgument() {
+        return true;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-vxquery/blob/c74c9865/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/rules/RemoveRedundantCastExpressionsRule.java
----------------------------------------------------------------------
diff --git a/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/rules/RemoveRedundantCastExpressionsRule.java b/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/rules/RemoveRedundantCastExpressionsRule.java
new file mode 100644
index 0000000..bf06b00
--- /dev/null
+++ b/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/rules/RemoveRedundantCastExpressionsRule.java
@@ -0,0 +1,61 @@
+/*
+ * 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.vxquery.functions.BuiltinOperators;
+import org.apache.vxquery.types.SequenceType;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+
+/**
+ * The rule searches for where the xquery cast function is used. When the
+ * expression's return type matches the treat expression type, the cast is
+ * removed.
+ * 
+ * <pre>
+ * Before
+ * 
+ *   plan__parent
+ *   %OPERATOR( $v1 : cast( \@input_expression, \@type_expression ) )
+ *   plan__child
+ *   
+ *   Where treat \@type_expression is the same as the return type of \@input_expression.
+ *   
+ * After 
+ * 
+ *   plan__parent
+ *   %OPERATOR( $v1 : \@input_expression )
+ *   plan__child
+ * </pre>
+ * 
+ * @author prestonc
+ */
+
+public class RemoveRedundantCastExpressionsRule extends AbstractRemoveRedundantTypeExpressionsRule {
+    protected FunctionIdentifier getSearchFunction() {
+        return BuiltinOperators.CAST.getFunctionIdentifier();
+    }
+
+    @Override
+    public boolean safeToReplace(SequenceType sTypeArg, SequenceType sTypeOutput) {
+        if (sTypeArg != null && sTypeOutput != null && sTypeOutput.equals(sTypeArg)) {
+            // Same type.
+            return true;
+        }
+        return false;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-vxquery/blob/c74c9865/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/rules/RemoveRedundantDataExpressionsRule.java
----------------------------------------------------------------------
diff --git a/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/rules/RemoveRedundantDataExpressionsRule.java b/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/rules/RemoveRedundantDataExpressionsRule.java
index d9554c4..fb05e4d 100644
--- a/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/rules/RemoveRedundantDataExpressionsRule.java
+++ b/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/rules/RemoveRedundantDataExpressionsRule.java
@@ -16,23 +16,10 @@
  */
 package org.apache.vxquery.compiler.rewriter.rules;
 
-import java.util.List;
-
-import org.apache.commons.lang3.mutable.Mutable;
-import org.apache.vxquery.compiler.rewriter.rules.util.ExpressionToolbox;
-import org.apache.vxquery.compiler.rewriter.rules.util.OperatorToolbox;
-import org.apache.vxquery.context.RootStaticContextImpl;
-import org.apache.vxquery.context.StaticContextImpl;
 import org.apache.vxquery.functions.BuiltinFunctions;
-import org.apache.vxquery.functions.Function;
 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.expressions.AbstractFunctionCallExpression;
-import edu.uci.ics.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+import edu.uci.ics.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 
 /**
  * The rule searches for where the xquery data function is used. When the
@@ -56,54 +43,20 @@ import edu.uci.ics.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
  * 
  * @author prestonc
  */
-public class RemoveRedundantDataExpressionsRule implements IAlgebraicRewriteRule {
-    final StaticContextImpl dCtx = new StaticContextImpl(RootStaticContextImpl.INSTANCE);
-    final int ARG_DATA = 0;
-
-    @Override
-    public boolean rewritePre(Mutable<ILogicalOperator> opRef, IOptimizationContext context) throws AlgebricksException {
-        return false;
+public class RemoveRedundantDataExpressionsRule extends AbstractRemoveRedundantTypeExpressionsRule {
+    protected FunctionIdentifier getSearchFunction() {
+        return BuiltinFunctions.FN_DATA_1.getFunctionIdentifier();
     }
 
     @Override
-    public boolean rewritePost(Mutable<ILogicalOperator> opRef, IOptimizationContext context)
-            throws AlgebricksException {
-        boolean modified = false;
-        List<Mutable<ILogicalExpression>> expressions = OperatorToolbox.getExpression(opRef);
-        for (Mutable<ILogicalExpression> expression : expressions) {
-            if (processDataExpression(expression)) {
-                modified = true;
-            }
-        }
-        return modified;
+    public boolean hasTypeArgument() {
+        return false;
     }
 
-    private boolean processDataExpression(Mutable<ILogicalExpression> search) {
-        boolean modified = false;
-        Mutable<ILogicalExpression> dataM = ExpressionToolbox.findFunctionExpression(search,
-                BuiltinFunctions.FN_DATA_1.getFunctionIdentifier());
-        if (dataM != null) {
-            // Get input function
-            AbstractFunctionCallExpression promoteFunction = (AbstractFunctionCallExpression) dataM.getValue();
-            Mutable<ILogicalExpression> argDataM = promoteFunction.getArguments().get(ARG_DATA);
-
-            // Find the input return type.
-            SequenceType inputSequenceType = null;
-            Function function = ExpressionToolbox.getBuiltIn(argDataM);
-            if (function == null) {
-                return false;
-            } else {
-                inputSequenceType = function.getSignature().getReturnType();
-            }
-
-            // remove
-            if (inputSequenceType != null && inputSequenceType.getItemType().isAtomicType()) {
-                dataM.setValue(argDataM.getValue());
-                modified = true;
-                processDataExpression(argDataM);
-            }
+    public boolean safeToReplace(SequenceType sTypeArg, SequenceType sTypeOutput) {
+        if (sTypeOutput != null && sTypeOutput.getItemType().isAtomicType()) {
+            return true;
         }
-        return modified;
+        return false;
     }
-
 }

http://git-wip-us.apache.org/repos/asf/incubator-vxquery/blob/c74c9865/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/rules/RemoveRedundantPromoteExpressionsRule.java
----------------------------------------------------------------------
diff --git a/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/rules/RemoveRedundantPromoteExpressionsRule.java b/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/rules/RemoveRedundantPromoteExpressionsRule.java
index 81b71ae..589618d 100644
--- a/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/rules/RemoveRedundantPromoteExpressionsRule.java
+++ b/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/rules/RemoveRedundantPromoteExpressionsRule.java
@@ -16,27 +16,14 @@
  */
 package org.apache.vxquery.compiler.rewriter.rules;
 
-import java.util.List;
-
-import org.apache.commons.lang3.mutable.Mutable;
-import org.apache.vxquery.compiler.rewriter.rules.util.ExpressionToolbox;
-import org.apache.vxquery.compiler.rewriter.rules.util.OperatorToolbox;
-import org.apache.vxquery.context.RootStaticContextImpl;
-import org.apache.vxquery.context.StaticContextImpl;
-import org.apache.vxquery.datamodel.accessors.TaggedValuePointable;
+import org.apache.vxquery.datamodel.values.ValueTag;
 import org.apache.vxquery.functions.BuiltinOperators;
-import org.apache.vxquery.functions.Function;
+import org.apache.vxquery.runtime.functions.cast.CastToDoubleOperation;
+import org.apache.vxquery.runtime.functions.cast.CastToFloatOperation;
+import org.apache.vxquery.types.BuiltinTypeRegistry;
 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.expressions.AbstractFunctionCallExpression;
-import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.ConstantExpression;
-import edu.uci.ics.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
-import edu.uci.ics.hyracks.data.std.primitive.IntegerPointable;
+import edu.uci.ics.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 
 /**
  * The rule searches for where the xquery promote function is used. When the
@@ -62,67 +49,26 @@ import edu.uci.ics.hyracks.data.std.primitive.IntegerPointable;
  * @author prestonc
  */
 
-public class RemoveRedundantPromoteExpressionsRule implements IAlgebraicRewriteRule {
-    final StaticContextImpl dCtx = new StaticContextImpl(RootStaticContextImpl.INSTANCE);
-    final int ARG_DATA = 0;
-    final int ARG_TYPE = 1;
-
+public class RemoveRedundantPromoteExpressionsRule extends AbstractRemoveRedundantTypeExpressionsRule {
     @Override
-    public boolean rewritePre(Mutable<ILogicalOperator> opRef, IOptimizationContext context) throws AlgebricksException {
-        return false;
+    protected FunctionIdentifier getSearchFunction() {
+        return BuiltinOperators.PROMOTE.getFunctionIdentifier();
     }
 
     @Override
-    public boolean rewritePost(Mutable<ILogicalOperator> opRef, IOptimizationContext context)
-            throws AlgebricksException {
-        boolean modified = false;
-        List<Mutable<ILogicalExpression>> expressions = OperatorToolbox.getExpression(opRef);
-        for (Mutable<ILogicalExpression> expression : expressions) {
-            if (processPromoteExpression(expression)) {
-                modified = true;
+    public boolean safeToReplace(SequenceType sTypeArg, SequenceType sTypeOutput) {
+        if (sTypeArg != null) {
+            if (sTypeArg.getItemType() != BuiltinTypeRegistry.XS_DOUBLE
+                    && sTypeArg.getItemType() != BuiltinTypeRegistry.XS_FLOAT
+                    && sTypeArg.getItemType() != BuiltinTypeRegistry.XS_STRING) {
+                // These types can not be promoted.
+                return true;
             }
-        }
-        return modified;
-    }
-
-    private boolean processPromoteExpression(Mutable<ILogicalExpression> search) {
-        boolean modified = false;
-        Mutable<ILogicalExpression> promoteM = ExpressionToolbox.findFunctionExpression(search,
-                BuiltinOperators.PROMOTE.getFunctionIdentifier());
-        if (promoteM != null) {
-            // Get input function
-            AbstractFunctionCallExpression promoteFunction = (AbstractFunctionCallExpression) promoteM.getValue();
-            Mutable<ILogicalExpression> argDataM = promoteFunction.getArguments().get(ARG_DATA);
-            
-            // Find the input return type.
-            SequenceType inputSequenceType = null;
-            Function function = ExpressionToolbox.getBuiltIn(argDataM);
-            if (function == null) {
-                return false;
-            } else {
-                inputSequenceType = function.getSignature().getReturnType();
-            }
-
-            // Find the promote type.
-            ILogicalExpression argType = promoteFunction.getArguments().get(ARG_TYPE).getValue();
-            if (argType.getExpressionTag() != LogicalExpressionTag.CONSTANT) {
-                return false;
-            }
-            TaggedValuePointable tvp = new TaggedValuePointable();
-            ExpressionToolbox.getConstantAsPointable((ConstantExpression) argType, tvp);
-
-            IntegerPointable pTypeCode = (IntegerPointable) IntegerPointable.FACTORY.createPointable();
-            tvp.getValue(pTypeCode);
-            SequenceType sType = dCtx.lookupSequenceType(pTypeCode.getInteger());
-
-            // remove
-            if (inputSequenceType != null && inputSequenceType.equals(sType)) {
-                promoteM.setValue(argDataM.getValue());
-                modified = true;
-                processPromoteExpression(argDataM);
+            if (sTypeOutput != null && sTypeOutput.equals(sTypeArg)) {
+                // Same type.
+                return true;
             }
         }
-        return modified;
+        return false;
     }
-
 }

http://git-wip-us.apache.org/repos/asf/incubator-vxquery/blob/c74c9865/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/rules/RemoveRedundantTreatExpressionsRule.java
----------------------------------------------------------------------
diff --git a/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/rules/RemoveRedundantTreatExpressionsRule.java b/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/rules/RemoveRedundantTreatExpressionsRule.java
index 67c3d5e..63ad06f 100644
--- a/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/rules/RemoveRedundantTreatExpressionsRule.java
+++ b/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/rules/RemoveRedundantTreatExpressionsRule.java
@@ -16,27 +16,11 @@
  */
 package org.apache.vxquery.compiler.rewriter.rules;
 
-import java.util.List;
-
-import org.apache.commons.lang3.mutable.Mutable;
-import org.apache.vxquery.compiler.rewriter.rules.util.ExpressionToolbox;
-import org.apache.vxquery.compiler.rewriter.rules.util.OperatorToolbox;
-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.runtime.functions.type.SequenceTypeMatcher;
 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.expressions.AbstractFunctionCallExpression;
-import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.ConstantExpression;
-import edu.uci.ics.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
-import edu.uci.ics.hyracks.data.std.primitive.IntegerPointable;
+import edu.uci.ics.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 
 /**
  * The rule searches for where the xquery treat function is used. When the
@@ -62,67 +46,22 @@ import edu.uci.ics.hyracks.data.std.primitive.IntegerPointable;
  * @author prestonc
  */
 
-public class RemoveRedundantTreatExpressionsRule implements IAlgebraicRewriteRule {
-    final StaticContextImpl dCtx = new StaticContextImpl(RootStaticContextImpl.INSTANCE);
-    final int ARG_DATA = 0;
-    final int ARG_TYPE = 1;
+public class RemoveRedundantTreatExpressionsRule extends AbstractRemoveRedundantTypeExpressionsRule {
+    final SequenceTypeMatcher stm = new SequenceTypeMatcher();
 
-    @Override
-    public boolean rewritePre(Mutable<ILogicalOperator> opRef, IOptimizationContext context) throws AlgebricksException {
-        return false;
+    protected FunctionIdentifier getSearchFunction() {
+        return BuiltinOperators.TREAT.getFunctionIdentifier();
     }
 
     @Override
-    public boolean rewritePost(Mutable<ILogicalOperator> opRef, IOptimizationContext context)
-            throws AlgebricksException {
-        boolean modified = false;
-        List<Mutable<ILogicalExpression>> expressions = OperatorToolbox.getExpression(opRef);
-        for (Mutable<ILogicalExpression> expression : expressions) {
-            if (processTreatExpression(expression)) {
-                modified = true;
-            }
-        }
-        return modified;
-    }
-
-    private boolean processTreatExpression(Mutable<ILogicalExpression> search) {
-        boolean modified = false;
-        Mutable<ILogicalExpression> treatM = ExpressionToolbox.findFunctionExpression(search,
-                BuiltinOperators.TREAT.getFunctionIdentifier());
-        if (treatM != null) {
-            // Get input function
-            AbstractFunctionCallExpression treatFunction = (AbstractFunctionCallExpression) treatM.getValue();
-            Mutable<ILogicalExpression> argDataM = treatFunction.getArguments().get(ARG_DATA);
-            
-            // Find the input return type.
-            SequenceType inputSequenceType = null;
-            Function function = ExpressionToolbox.getBuiltIn(argDataM);
-            if (function == null) {
-                return false;
-            } else {
-                inputSequenceType = function.getSignature().getReturnType();
-            }
-
-            // Find the treat type.
-            ILogicalExpression argType = treatFunction.getArguments().get(ARG_TYPE).getValue();
-            if (argType.getExpressionTag() != LogicalExpressionTag.CONSTANT) {
-                return false;
-            }
-            TaggedValuePointable tvp = new TaggedValuePointable();
-            ExpressionToolbox.getConstantAsPointable((ConstantExpression) argType, tvp);
-
-            IntegerPointable pTypeCode = (IntegerPointable) IntegerPointable.FACTORY.createPointable();
-            tvp.getValue(pTypeCode);
-            SequenceType sType = dCtx.lookupSequenceType(pTypeCode.getInteger());
-
-            // remove
-            if (inputSequenceType != null && inputSequenceType.equals(sType)) {
-                treatM.setValue(argDataM.getValue());
-                modified = true;
-                processTreatExpression(argDataM);
+    public boolean safeToReplace(SequenceType sTypeArg, SequenceType sTypeOutput) {
+        if (sTypeArg != null) {
+            stm.setSequenceType(sTypeArg);
+            if (sTypeOutput != null && stm.isSubType(sTypeOutput)) {
+                // Same type.
+                return true;
             }
         }
-        return modified;
+        return false;
     }
-
 }

http://git-wip-us.apache.org/repos/asf/incubator-vxquery/blob/c74c9865/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/rules/util/ExpressionToolbox.java
----------------------------------------------------------------------
diff --git a/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/rules/util/ExpressionToolbox.java b/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/rules/util/ExpressionToolbox.java
index e494f0b..7d54068 100644
--- a/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/rules/util/ExpressionToolbox.java
+++ b/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/rules/util/ExpressionToolbox.java
@@ -16,23 +16,33 @@
  */
 package org.apache.vxquery.compiler.rewriter.rules.util;
 
+import java.util.List;
+
 import org.apache.commons.lang3.mutable.Mutable;
 import org.apache.vxquery.compiler.algebricks.VXQueryConstantValue;
+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.types.AnyNodeType;
+import org.apache.vxquery.types.Quantifier;
+import org.apache.vxquery.types.SequenceType;
 
 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.LogicalExpressionTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
 import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
 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.expressions.VariableReferenceExpression;
 import edu.uci.ics.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
+import edu.uci.ics.hyracks.data.std.primitive.IntegerPointable;
 
 public class ExpressionToolbox {
-
     public static Mutable<ILogicalExpression> findVariableExpression(Mutable<ILogicalExpression> mutableLe,
             LogicalVariable lv) {
         ILogicalExpression le = mutableLe.getValue();
@@ -88,7 +98,7 @@ public class ExpressionToolbox {
         return null;
     }
 
-    public static Mutable<ILogicalExpression> findFunctionExpression(Mutable<ILogicalExpression> mutableLe,
+    public static Mutable<ILogicalExpression> findFirstFunctionExpression(Mutable<ILogicalExpression> mutableLe,
             FunctionIdentifier fi) {
         ILogicalExpression le = mutableLe.getValue();
         if (le.getExpressionTag() == LogicalExpressionTag.FUNCTION_CALL) {
@@ -97,7 +107,7 @@ public class ExpressionToolbox {
                 return mutableLe;
             }
             for (Mutable<ILogicalExpression> argExp : afce.getArguments()) {
-                Mutable<ILogicalExpression> resultLe = findFunctionExpression(argExp, fi);
+                Mutable<ILogicalExpression> resultLe = findFirstFunctionExpression(argExp, fi);
                 if (resultLe != null) {
                     return resultLe;
                 }
@@ -106,6 +116,23 @@ public class ExpressionToolbox {
         return null;
     }
 
+    /**
+     * Find all functions for a specific expression.
+     */
+    public static void findAllFunctionExpressions(Mutable<ILogicalExpression> mutableLe, FunctionIdentifier fi,
+            List<Mutable<ILogicalExpression>> finds) {
+        ILogicalExpression le = mutableLe.getValue();
+        if (le.getExpressionTag() == LogicalExpressionTag.FUNCTION_CALL) {
+            AbstractFunctionCallExpression afce = (AbstractFunctionCallExpression) le;
+            if (afce.getFunctionIdentifier().equals(fi)) {
+                finds.add(mutableLe);
+            }
+            for (Mutable<ILogicalExpression> argExp : afce.getArguments()) {
+                findAllFunctionExpressions(argExp, fi, finds);
+            }
+        }
+    }
+
     public static Function getBuiltIn(Mutable<ILogicalExpression> mutableLe) {
         ILogicalExpression le = mutableLe.getValue();
         if (le.getExpressionTag() == LogicalExpressionTag.FUNCTION_CALL) {
@@ -129,4 +156,63 @@ public class ExpressionToolbox {
         tvp.set(treatTypeConstant.getValue(), 0, treatTypeConstant.getValue().length);
     }
 
+    public static int getTypeExpressionTypeArgument(Mutable<ILogicalExpression> searchM) {
+        final int ARG_TYPE = 1;
+        AbstractFunctionCallExpression searchFunction = (AbstractFunctionCallExpression) searchM.getValue();
+        ILogicalExpression argType = searchFunction.getArguments().get(ARG_TYPE).getValue();
+        if (argType.getExpressionTag() != LogicalExpressionTag.CONSTANT) {
+            return -1;
+        }
+        TaggedValuePointable tvp = (TaggedValuePointable) TaggedValuePointable.FACTORY.createPointable();
+        ExpressionToolbox.getConstantAsPointable((ConstantExpression) argType, tvp);
+
+        IntegerPointable pTypeCode = (IntegerPointable) IntegerPointable.FACTORY.createPointable();
+        tvp.getValue(pTypeCode);
+        return pTypeCode.getInteger();
+    }
+
+    public static SequenceType getTypeExpressionTypeArguement(Mutable<ILogicalExpression> searchM,
+            StaticContextImpl dCtx) {
+        int typeId = getTypeExpressionTypeArgument(searchM);
+        if (typeId > 0) {
+            return dCtx.lookupSequenceType(typeId);
+        } else {
+            return null;
+        }
+    }
+
+    public static SequenceType getOutputSequenceType(Mutable<ILogicalOperator> opRef,
+            Mutable<ILogicalExpression> argFirstM, StaticContextImpl dCtx) {
+        ILogicalExpression argFirstLe = argFirstM.getValue();
+        switch (argFirstLe.getExpressionTag()) {
+            case FUNCTION_CALL:
+                // Only process defined functions.
+                Function function = ExpressionToolbox.getBuiltIn(argFirstM);
+                if (function == null) {
+                    return null;
+                } else if (function.getFunctionIdentifier().equals(BuiltinOperators.CAST.getFunctionIdentifier())) {
+                    // Special case since case has multiple type outputs.
+                    return ExpressionToolbox.getTypeExpressionTypeArguement(argFirstM, dCtx);
+                } else {
+                    return function.getSignature().getReturnType();
+                }
+            case CONSTANT:
+                // Consider constant values.
+                ConstantExpression constantExpression = (ConstantExpression) argFirstLe;
+                VXQueryConstantValue constantValue = (VXQueryConstantValue) constantExpression.getValue();
+                return constantValue.getType();
+            case VARIABLE:
+                VariableReferenceExpression variableRefExp = (VariableReferenceExpression) argFirstLe;
+                LogicalVariable variableId = variableRefExp.getVariableReference();
+                Mutable<ILogicalOperator> variableProducer = OperatorToolbox.findProducerOf(opRef, variableId);
+                if (variableProducer == null) {
+                    return null;
+                }
+                AbstractLogicalOperator variableOp = (AbstractLogicalOperator) variableProducer.getValue();
+                if (variableOp.getOperatorTag() == LogicalOperatorTag.DATASOURCESCAN) {
+                    return SequenceType.create(AnyNodeType.INSTANCE, Quantifier.QUANT_ONE);
+                }
+        }
+        return null;
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-vxquery/blob/c74c9865/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/rules/util/OperatorToolbox.java
----------------------------------------------------------------------
diff --git a/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/rules/util/OperatorToolbox.java b/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/rules/util/OperatorToolbox.java
index fd1f501..da85f2d 100644
--- a/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/rules/util/OperatorToolbox.java
+++ b/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/rules/util/OperatorToolbox.java
@@ -16,8 +16,8 @@
  */
 package org.apache.vxquery.compiler.rewriter.rules.util;
 
-import java.util.List;
 import java.util.ArrayList;
+import java.util.List;
 
 import org.apache.commons.lang3.mutable.Mutable;
 
@@ -26,8 +26,11 @@ import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
 import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractAssignOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractBinaryJoinOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractScanOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractUnnestOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.SelectOperator;
 
 public class OperatorToolbox {
 
@@ -44,7 +47,7 @@ public class OperatorToolbox {
         }
         return opRef;
     }
-    
+
     public static AbstractLogicalOperator findLastSubplanOperator(AbstractLogicalOperator op) {
         AbstractLogicalOperator next;
         while (op.getOperatorTag() != LogicalOperatorTag.NESTEDTUPLESOURCE) {
@@ -56,8 +59,8 @@ public class OperatorToolbox {
         }
         return op;
     }
-    
-    public static List<Mutable<ILogicalExpression>> getExpression(Mutable<ILogicalOperator> opRef) {
+
+    public static List<Mutable<ILogicalExpression>> getExpressions(Mutable<ILogicalOperator> opRef) {
         AbstractLogicalOperator op = (AbstractLogicalOperator) opRef.getValue();
         List<Mutable<ILogicalExpression>> result = new ArrayList<Mutable<ILogicalExpression>>();
         switch (op.getOperatorTag()) {
@@ -67,6 +70,15 @@ public class OperatorToolbox {
                 AbstractAssignOperator aao = (AbstractAssignOperator) op;
                 result.addAll(aao.getExpressions());
                 break;
+            case INNERJOIN:
+            case LEFTOUTERJOIN:
+                AbstractBinaryJoinOperator abjo = (AbstractBinaryJoinOperator) op;
+                result.add(abjo.getCondition());
+                break;
+            case SELECT:
+                SelectOperator so = (SelectOperator) op;
+                result.add(so.getCondition());
+                break;
             case UNNEST:
             case UNNEST_MAP:
                 AbstractUnnestOperator auo = (AbstractUnnestOperator) op;
@@ -81,6 +93,51 @@ public class OperatorToolbox {
             case EXTENSION_OPERATOR:
             case GROUP:
             case INDEX_INSERT_DELETE:
+            case INSERT_DELETE:
+            case LIMIT:
+            case NESTEDTUPLESOURCE:
+            case ORDER:
+            case PARTITIONINGSPLIT:
+            case PROJECT:
+            case REPLICATE:
+            case SCRIPT:
+            case SINK:
+            case SUBPLAN:
+            case UNIONALL:
+            case UPDATE:
+            case WRITE:
+            case WRITE_RESULT:
+            default:
+                // TODO Not yet implemented.
+                break;
+        }
+        return result;
+    }
+
+    public static Mutable<ILogicalExpression> getExpressionOf(Mutable<ILogicalOperator> opRef, LogicalVariable lv) {
+        AbstractLogicalOperator op = (AbstractLogicalOperator) opRef.getValue();
+        switch (op.getOperatorTag()) {
+            case AGGREGATE:
+            case ASSIGN:
+            case RUNNINGAGGREGATE:
+                AbstractAssignOperator aao = (AbstractAssignOperator) op;
+                if (!aao.getVariables().contains(lv)) {
+                    return null;
+                }
+                return aao.getExpressions().get(aao.getVariables().indexOf(lv));
+            case UNNEST:
+            case UNNEST_MAP:
+                AbstractUnnestOperator ano = (AbstractUnnestOperator) op;
+                return ano.getExpressionRef();
+            case CLUSTER:
+            case DATASOURCESCAN:
+            case DISTINCT:
+            case DISTRIBUTE_RESULT:
+            case EMPTYTUPLESOURCE:
+            case EXCHANGE:
+            case EXTENSION_OPERATOR:
+            case GROUP:
+            case INDEX_INSERT_DELETE:
             case INNERJOIN:
             case INSERT_DELETE:
             case LEFTOUTERJOIN:
@@ -99,12 +156,12 @@ public class OperatorToolbox {
             case WRITE:
             case WRITE_RESULT:
             default:
+                // TODO Not yet implemented.
                 break;
         }
-        return result;
+        return null;
     }
 
-
     public static Mutable<ILogicalOperator> findProducerOf(Mutable<ILogicalOperator> opRef, LogicalVariable lv) {
         AbstractLogicalOperator op = (AbstractLogicalOperator) opRef.getValue();
         switch (op.getOperatorTag()) {
@@ -122,15 +179,16 @@ public class OperatorToolbox {
                     }
                 }
                 break;
+            case DATASOURCESCAN:
             case UNNEST:
             case UNNEST_MAP:
-                AbstractUnnestOperator auo = (AbstractUnnestOperator) op;
-                if (auo.getVariables().contains(lv)) {
+                AbstractScanOperator aso = (AbstractScanOperator) op;
+                if (aso.getVariables().contains(lv)) {
                     return opRef;
                 }
                 for (Mutable<ILogicalOperator> input : op.getInputs()) {
                     Mutable<ILogicalOperator> opInput = findProducerOf(input, lv);
-                     if (opInput != null) {
+                    if (opInput != null) {
                         return opInput;
                     }
                 }
@@ -139,7 +197,6 @@ public class OperatorToolbox {
             case NESTEDTUPLESOURCE:
                 return null;
             case CLUSTER:
-            case DATASOURCESCAN:
             case DISTINCT:
             case DISTRIBUTE_RESULT:
             case EXCHANGE:
@@ -163,7 +220,13 @@ public class OperatorToolbox {
             case WRITE:
             case WRITE_RESULT:
             default:
-                // TODO Not yet implemented.
+                // Skip operators and go look at input.
+                for (Mutable<ILogicalOperator> input : op.getInputs()) {
+                    Mutable<ILogicalOperator> opInput = findProducerOf(input, lv);
+                    if (opInput != null) {
+                        return opInput;
+                    }
+                }
                 break;
         }
         return null;

http://git-wip-us.apache.org/repos/asf/incubator-vxquery/blob/c74c9865/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/type/SequenceTypeMatcher.java
----------------------------------------------------------------------
diff --git a/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/type/SequenceTypeMatcher.java b/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/type/SequenceTypeMatcher.java
index 974abd1..5142ac4 100644
--- a/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/type/SequenceTypeMatcher.java
+++ b/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/type/SequenceTypeMatcher.java
@@ -153,6 +153,36 @@ public class SequenceTypeMatcher {
         return false;
     }
 
+    public boolean isSubType(SequenceType testST) {
+        Quantifier stq = sequenceType.getQuantifier();
+        ItemType it = sequenceType.getItemType();
+        if (stq.isSubQuantifier(testST.getQuantifier())) {
+            if (it instanceof AnyItemType) {
+                return true;
+            } else if (it.isAtomicType() && testST.getItemType().isAtomicType()) {
+                AtomicType ait = (AtomicType) it;
+                AtomicType testIT = (AtomicType) testST.getItemType();
+                if (BuiltinTypeRegistry.INSTANCE.isBuiltinTypeId(testIT.getTypeId())) {
+                    SchemaType vType = BuiltinTypeRegistry.INSTANCE.getSchemaTypeById(testIT.getTypeId());
+                    while (vType != null && vType.getTypeId() != ait.getTypeId()) {
+                        vType = vType.getBaseType();
+                    }
+                    return vType != null;
+                }
+            } else if (it instanceof NodeType && testST.getItemType() instanceof NodeType) {
+                NodeType nt = (NodeType) it;
+                NodeKind kind = nt.getNodeKind();
+                NodeType testNT = (NodeType) testST.getItemType();
+                NodeKind testKind = testNT.getNodeKind();
+                if (kind == NodeKind.ANY || kind == testKind) {
+                    return true;
+                }
+            }
+            return false;
+        }
+        return false;
+    }
+
     private Quantifier getSequenceQuantifier(SequencePointable seqp) {
         switch (seqp.getEntryCount()) {
             case 0:
@@ -167,7 +197,7 @@ public class SequenceTypeMatcher {
     public void setSequenceType(SequenceType sType) {
         this.sequenceType = sType;
     }
-    
+
     public String toString() {
         return "sequenceMatcher[" + this.sequenceType + "]";
     }