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 2015/08/14 00:07:53 UTC

[3/3] vxquery git commit: Unnesting rule and code additions to support XMark Query 8.

Unnesting rule and code additions to support XMark Query 8.


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

Branch: refs/heads/master
Commit: 7fc0d93b4f83fa9ead1f053bf041e1b871ee9f33
Parents: 3e40e5a
Author: Preston Carman <pr...@apache.org>
Authored: Thu Aug 13 14:48:17 2015 -0700
Committer: Preston Carman <pr...@apache.org>
Committed: Thu Aug 13 14:48:17 2015 -0700

----------------------------------------------------------------------
 .../java/org/apache/vxquery/cli/VXQuery.java    |   54 +-
 .../compiler/rewriter/RewriteRuleset.java       |   50 +-
 .../rewriter/rules/ConsolidateUnnestsRule.java  |   79 +-
 .../rules/ConvertAssignToUnnestRule.java        |   31 +-
 .../ConvertFromAlgebricksExpressionsRule.java   |   33 +-
 .../ConvertToAlgebricksExpressionsRule.java     |   16 +-
 .../rules/InlineNestedVariablesRule.java        |   97 --
 .../ExtractFunctionsFromJoinConditionRule.java  |  141 ++
 .../InlineNestedVariablesRule.java              |   97 ++
 ...oveFreeVariableOperatorOutOfSubplanRule.java |  178 +++
 ...GeneralComparisonScalarEvaluatorFactory.java |   12 +-
 .../IfThenElseScalarEvaluatorFactory.java       |   37 +-
 .../vxquery/serializer/XMLSerializer.java       |    9 +-
 .../xmlquery/query/XMLQueryCompiler.java        |   10 +-
 .../xmlquery/translator/XMLQueryTranslator.java |   42 +-
 vxquery-xtest/pom.xml                           |    1 +
 vxquery-xtest/results/xqts.txt                  | 1480 +++++++++---------
 .../org/apache/vxquery/xtest/XTestOptions.java  |   38 +-
 .../vxquery/xtest/AbstractXQueryTest.java       |    6 +-
 .../org/apache/vxquery/xtest/VXQueryTest.java   |    4 +-
 .../apache/vxquery/xtest/VXQueryXMarkTest.java  |   55 +
 .../org/apache/vxquery/xtest/XMarkTest.java     |    5 +-
 .../java/org/apache/vxquery/xtest/XQTSTest.java |    2 +-
 .../Queries/XQuery/XMark/Modified/q08.xq        |    2 +-
 24 files changed, 1465 insertions(+), 1014 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/vxquery/blob/7fc0d93b/vxquery-cli/src/main/java/org/apache/vxquery/cli/VXQuery.java
----------------------------------------------------------------------
diff --git a/vxquery-cli/src/main/java/org/apache/vxquery/cli/VXQuery.java b/vxquery-cli/src/main/java/org/apache/vxquery/cli/VXQuery.java
index 5b99978..cf21c76 100644
--- a/vxquery-cli/src/main/java/org/apache/vxquery/cli/VXQuery.java
+++ b/vxquery-cli/src/main/java/org/apache/vxquery/cli/VXQuery.java
@@ -77,7 +77,7 @@ public class VXQuery {
 
     /**
      * Constructor to use command line options passed.
-     * 
+     *
      * @param opts
      *            Command line options object
      */
@@ -87,7 +87,7 @@ public class VXQuery {
 
     /**
      * Main method to get command line options and execute query process.
-     * 
+     *
      * @param args
      * @throws Exception
      */
@@ -134,7 +134,7 @@ public class VXQuery {
     /**
      * Creates a new Hyracks connection with: the client IP address and port provided, if IP address is provided in command line. Otherwise create a new virtual
      * cluster with Hyracks nodes. Queries passed are run either way. After running queries, if a virtual cluster has been created, it is shut down.
-     * 
+     *
      * @throws Exception
      */
     private void execute() throws Exception {
@@ -160,7 +160,7 @@ public class VXQuery {
     /**
      * Reads the contents of the files passed in the list of arguments to a string. If -showquery argument is passed, output the query as string. Run the query
      * for the string.
-     * 
+     *
      * @throws IOException
      * @throws SystemException
      * @throws Exception
@@ -226,7 +226,7 @@ public class VXQuery {
 
     /**
      * Get cluster node configuration.
-     * 
+     *
      * @return Configuration of node controllers as array of Strings.
      * @throws Exception
      */
@@ -243,7 +243,7 @@ public class VXQuery {
     /**
      * Creates a Hyracks dataset, if not already existing with the job frame size, and 1 reader. Allocates a new buffer of size specified in the frame of Hyracks
      * node. Creates new dataset reader with the current job ID and result set ID. Outputs the string in buffer for each frame.
-     * 
+     *
      * @param spec
      *            JobSpecification object, containing frame size. Current specified job.
      * @param writer
@@ -274,7 +274,7 @@ public class VXQuery {
 
     /**
      * Create a unique result set id to get the correct query back from the cluster.
-     * 
+     *
      * @return Result Set id generated with current system time.
      */
     protected ResultSetId createResultSetId() {
@@ -284,7 +284,7 @@ public class VXQuery {
     /**
      * Start local virtual cluster with cluster controller node and node controller nodes. IP address provided for node controller is localhost. Unassigned ports
      * 39000 and 39001 are used for client and cluster port respectively. Creates a new Hyracks connection with the IP address and client ports.
-     * 
+     *
      * @throws Exception
      */
     public void startLocalHyracks() throws Exception {
@@ -317,7 +317,7 @@ public class VXQuery {
 
     /**
      * Shuts down the virtual cluster, along with all nodes and node execution, network and queue managers.
-     * 
+     *
      * @throws Exception
      */
     public void stopLocalHyracks() throws Exception {
@@ -329,7 +329,7 @@ public class VXQuery {
 
     /**
      * Reads the contents of file given in query into a String. The file is always closed. For XML files UTF-8 encoding is used.
-     * 
+     *
      * @param query
      *            The query with filename to be processed
      * @return UTF-8 formatted query string
@@ -341,7 +341,7 @@ public class VXQuery {
 
     /**
      * Save and print out the timing message.
-     * 
+     *
      * @param message
      */
     private static void timingMessage(String message) {
@@ -353,55 +353,55 @@ public class VXQuery {
      * Helper class with fields and methods to handle all command line options
      */
     private static class CmdLineOptions {
-        @Option(name = "-available-processors", usage = "Number of available processors. (default java's available processors)")
+        @Option(name = "-available-processors", usage = "Number of available processors. (default: java's available processors)")
         private int availableProcessors = -1;
 
-        @Option(name = "-client-net-ip-address", usage = "IP Address of the ClusterController")
+        @Option(name = "-client-net-ip-address", usage = "IP Address of the ClusterController.")
         private String clientNetIpAddress = null;
 
-        @Option(name = "-client-net-port", usage = "Port of the ClusterController (default 1098)")
+        @Option(name = "-client-net-port", usage = "Port of the ClusterController. (default: 1098)")
         private int clientNetPort = 1098;
 
-        @Option(name = "-local-node-controllers", usage = "Number of local node controllers (default 1)")
+        @Option(name = "-local-node-controllers", usage = "Number of local node controllers. (default: 1)")
         private int localNodeControllers = 1;
 
-        @Option(name = "-frame-size", usage = "Frame size in bytes. (default 65,536)")
+        @Option(name = "-frame-size", usage = "Frame size in bytes. (default: 65,536)")
         private int frameSize = 65536;
 
-        @Option(name = "-join-hash-size", usage = "Join hash size in bytes. (default 67,108,864)")
+        @Option(name = "-join-hash-size", usage = "Join hash size in bytes. (default: 67,108,864)")
         private long joinHashSize = -1;
 
-        @Option(name = "-maximum-data-size", usage = "Maximum possible data size in bytes. (default 150,323,855,000)")
+        @Option(name = "-maximum-data-size", usage = "Maximum possible data size in bytes. (default: 150,323,855,000)")
         private long maximumDataSize = -1;
 
         @Option(name = "-buffer-size", usage = "Disk read buffer size in bytes.")
         private int bufferSize = -1;
 
-        @Option(name = "-O", usage = "Optimization Level. Default: Full Optimization")
+        @Option(name = "-O", usage = "Optimization Level. (default: Full Optimization)")
         private int optimizationLevel = Integer.MAX_VALUE;
 
-        @Option(name = "-showquery", usage = "Show query string")
+        @Option(name = "-showquery", usage = "Show query string.")
         private boolean showQuery;
 
-        @Option(name = "-showast", usage = "Show abstract syntax tree")
+        @Option(name = "-showast", usage = "Show abstract syntax tree.")
         private boolean showAST;
 
-        @Option(name = "-showtet", usage = "Show translated expression tree")
+        @Option(name = "-showtet", usage = "Show translated expression tree.")
         private boolean showTET;
 
-        @Option(name = "-showoet", usage = "Show optimized expression tree")
+        @Option(name = "-showoet", usage = "Show optimized expression tree.")
         private boolean showOET;
 
-        @Option(name = "-showrp", usage = "Show Runtime plan")
+        @Option(name = "-showrp", usage = "Show Runtime plan.")
         private boolean showRP;
 
-        @Option(name = "-compileonly", usage = "Compile the query and stop")
+        @Option(name = "-compileonly", usage = "Compile the query and stop.")
         private boolean compileOnly;
 
-        @Option(name = "-repeatexec", usage = "Number of times to repeat execution")
+        @Option(name = "-repeatexec", usage = "Number of times to repeat execution.")
         private int repeatExec = 1;
 
-        @Option(name = "-timing", usage = "Produce timing information")
+        @Option(name = "-timing", usage = "Produce timing information.")
         private boolean timing;
 
         @Option(name = "-timing-ignore-queries", usage = "Ignore the first X number of quereies.")

http://git-wip-us.apache.org/repos/asf/vxquery/blob/7fc0d93b/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 7b55856..c3d0358 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
@@ -22,13 +22,11 @@ import java.util.List;
 import org.apache.vxquery.compiler.rewriter.rules.ConsolidateDescandantChild;
 import org.apache.vxquery.compiler.rewriter.rules.ConsolidateAssignAggregateRule;
 import org.apache.vxquery.compiler.rewriter.rules.ConvertAssignToUnnestRule;
-import org.apache.vxquery.compiler.rewriter.rules.ReplaceSourceMapInDocExpression;
 import org.apache.vxquery.compiler.rewriter.rules.ConvertFromAlgebricksExpressionsRule;
 import org.apache.vxquery.compiler.rewriter.rules.ConvertToAlgebricksExpressionsRule;
 import org.apache.vxquery.compiler.rewriter.rules.EliminateSubplanForSingleItemsRule;
 import org.apache.vxquery.compiler.rewriter.rules.EliminateUnnestAggregateSequencesRule;
 import org.apache.vxquery.compiler.rewriter.rules.EliminateUnnestAggregateSubplanRule;
-import org.apache.vxquery.compiler.rewriter.rules.InlineNestedVariablesRule;
 import org.apache.vxquery.compiler.rewriter.rules.IntroduceCollectionRule;
 import org.apache.vxquery.compiler.rewriter.rules.IntroduceTwoStepAggregateRule;
 import org.apache.vxquery.compiler.rewriter.rules.PushChildIntoDataScanRule;
@@ -40,17 +38,24 @@ import org.apache.vxquery.compiler.rewriter.rules.RemoveRedundantPromoteExpressi
 import org.apache.vxquery.compiler.rewriter.rules.RemoveRedundantTreatExpressionsRule;
 import org.apache.vxquery.compiler.rewriter.rules.RemoveUnusedSortDistinctNodesRule;
 import org.apache.vxquery.compiler.rewriter.rules.RemoveUnusedUnnestIterateRule;
+import org.apache.vxquery.compiler.rewriter.rules.ReplaceSourceMapInDocExpression;
 import org.apache.vxquery.compiler.rewriter.rules.SetCollectionDataSourceRule;
 import org.apache.vxquery.compiler.rewriter.rules.SetVariableIdContextRule;
+import org.apache.vxquery.compiler.rewriter.rules.algebricksalternatives.ExtractFunctionsFromJoinConditionRule;
+import org.apache.vxquery.compiler.rewriter.rules.algebricksalternatives.InlineNestedVariablesRule;
+import org.apache.vxquery.compiler.rewriter.rules.algebricksalternatives.MoveFreeVariableOperatorOutOfSubplanRule;
 
 import edu.uci.ics.hyracks.algebricks.core.rewriter.base.HeuristicOptimizer;
 import edu.uci.ics.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
 import edu.uci.ics.hyracks.algebricks.rewriter.rules.BreakSelectIntoConjunctsRule;
 import edu.uci.ics.hyracks.algebricks.rewriter.rules.ComplexJoinInferenceRule;
+import edu.uci.ics.hyracks.algebricks.rewriter.rules.ComplexUnnestToProductRule;
 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.EliminateGroupByEmptyKeyRule;
 import edu.uci.ics.hyracks.algebricks.rewriter.rules.EliminateSubplanRule;
+import edu.uci.ics.hyracks.algebricks.rewriter.rules.EliminateSubplanWithInputCardinalityOneRule;
 import edu.uci.ics.hyracks.algebricks.rewriter.rules.EnforceStructuralPropertiesRule;
 import edu.uci.ics.hyracks.algebricks.rewriter.rules.ExtractCommonOperatorsRule;
 import edu.uci.ics.hyracks.algebricks.rewriter.rules.ExtractGbyExpressionsRule;
@@ -58,17 +63,21 @@ import edu.uci.ics.hyracks.algebricks.rewriter.rules.FactorRedundantGroupAndDeco
 import edu.uci.ics.hyracks.algebricks.rewriter.rules.InferTypesRule;
 import edu.uci.ics.hyracks.algebricks.rewriter.rules.InlineAssignIntoAggregateRule;
 import edu.uci.ics.hyracks.algebricks.rewriter.rules.InlineVariablesRule;
+import edu.uci.ics.hyracks.algebricks.rewriter.rules.InsertOuterJoinRule;
+import edu.uci.ics.hyracks.algebricks.rewriter.rules.IntroJoinInsideSubplanRule;
 import edu.uci.ics.hyracks.algebricks.rewriter.rules.IntroduceAggregateCombinerRule;
 import edu.uci.ics.hyracks.algebricks.rewriter.rules.IntroduceGroupByCombinerRule;
+import edu.uci.ics.hyracks.algebricks.rewriter.rules.IntroduceGroupByForSubplanRule;
 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.MoveFreeVariableOperatorOutOfSubplanRule;
+import edu.uci.ics.hyracks.algebricks.rewriter.rules.NestedSubplanToJoinRule;
 import edu.uci.ics.hyracks.algebricks.rewriter.rules.PullSelectOutOfEqJoin;
 import edu.uci.ics.hyracks.algebricks.rewriter.rules.PushMapOperatorDownThroughProductRule;
 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;
 import edu.uci.ics.hyracks.algebricks.rewriter.rules.PushSelectIntoJoinRule;
+import edu.uci.ics.hyracks.algebricks.rewriter.rules.PushSubplanIntoGroupByRule;
 import edu.uci.ics.hyracks.algebricks.rewriter.rules.PushSubplanWithAggregateDownThroughProductRule;
 import edu.uci.ics.hyracks.algebricks.rewriter.rules.ReinferAllTypesRule;
 import edu.uci.ics.hyracks.algebricks.rewriter.rules.RemoveRedundantVariablesRule;
@@ -76,6 +85,7 @@ import edu.uci.ics.hyracks.algebricks.rewriter.rules.RemoveUnusedAssignAndAggreg
 import edu.uci.ics.hyracks.algebricks.rewriter.rules.SetAlgebricksPhysicalOperatorsRule;
 import edu.uci.ics.hyracks.algebricks.rewriter.rules.SetExecutionModeRule;
 import edu.uci.ics.hyracks.algebricks.rewriter.rules.SimpleUnnestToProductRule;
+import edu.uci.ics.hyracks.algebricks.rewriter.rules.SubplanOutOfGroupRule;
 
 public class RewriteRuleset {
     /**
@@ -98,7 +108,7 @@ public class RewriteRuleset {
         // TODO Fix the group by operator before putting back in the rule set.
         //        normalization.add(new ConvertAssignSortDistinctNodesToOperatorsRule());
 
-        // Find unnest followed by aggregate in a subplan. 
+        // Find unnest followed by aggregate in a subplan.
         normalization.add(new EliminateUnnestAggregateSubplanRule());
         normalization.add(new RemoveRedundantVariablesRule());
         normalization.add(new RemoveUnusedAssignAndAggregateRule());
@@ -199,6 +209,38 @@ public class RewriteRuleset {
         return typeInfer;
     }
 
+    /**
+     * Unnest more complex structures.
+     */
+    public final static List<IAlgebraicRewriteRule> buildUnnestingRuleCollection() {
+        List<IAlgebraicRewriteRule> xquery = new LinkedList<IAlgebraicRewriteRule>();
+
+        xquery.add(new PushSelectDownRule());
+        xquery.add(new SimpleUnnestToProductRule());
+        xquery.add(new ComplexUnnestToProductRule());
+        xquery.add(new ComplexJoinInferenceRule());
+        xquery.add(new PushSelectIntoJoinRule());
+        xquery.add(new IntroJoinInsideSubplanRule());
+        xquery.add(new PushMapOperatorDownThroughProductRule());
+        xquery.add(new PushSubplanWithAggregateDownThroughProductRule());
+        xquery.add(new IntroduceGroupByForSubplanRule());
+        xquery.add(new SubplanOutOfGroupRule());
+        xquery.add(new InsertOuterJoinRule());
+        xquery.add(new ExtractFunctionsFromJoinConditionRule());
+
+        xquery.add(new RemoveRedundantVariablesRule());
+        xquery.add(new RemoveUnusedAssignAndAggregateRule());
+
+        xquery.add(new FactorRedundantGroupAndDecorVarsRule());
+        xquery.add(new EliminateSubplanRule());
+        xquery.add(new EliminateGroupByEmptyKeyRule());
+        xquery.add(new PushSubplanIntoGroupByRule());
+        xquery.add(new NestedSubplanToJoinRule());
+        xquery.add(new EliminateSubplanWithInputCardinalityOneRule());
+
+        return xquery;
+    }
+
     public final static List<IAlgebraicRewriteRule> buildNormalizationRuleCollection() {
         List<IAlgebraicRewriteRule> normalization = new LinkedList<IAlgebraicRewriteRule>();
         normalization.add(new EliminateSubplanRule());

http://git-wip-us.apache.org/repos/asf/vxquery/blob/7fc0d93b/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/rules/ConsolidateUnnestsRule.java
----------------------------------------------------------------------
diff --git a/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/rules/ConsolidateUnnestsRule.java b/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/rules/ConsolidateUnnestsRule.java
index 6a2bb08..f1b2196 100644
--- a/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/rules/ConsolidateUnnestsRule.java
+++ b/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/rules/ConsolidateUnnestsRule.java
@@ -37,26 +37,26 @@ import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.UnnestOpera
  * operator (2). XQuery unnest operator (1) must have a scalar implementation
  * of the unnest function. If so the two unnest expressions can be merged
  * together.
- * 
+ *
  * <pre>
- * Before 
+ * Before
  * 
  *   plan__parent
  *   UNNEST( $v2 : uf2( $v1 ) )
  *   UNNEST( $v1 : uf1( $v0 ) )
  *   plan__child
- *   
+ * 
  *   Where $v1 is not used in plan__parent and uf1 is not a descendant expression.
- *   
+ * 
  * After
  * 
  *   plan__parent
  *   UNNEST( $v2 : uf2( sf1( $v0 ) ) )
  *   plan__child
- *   
+ * 
  *   uf1 becomes sf1 since it changes from unnesting to scalar expression.
  * </pre>
- * 
+ *
  * @author prestonc
  */
 public class ConsolidateUnnestsRule extends AbstractUsedVariablesProcessingRule {
@@ -75,41 +75,42 @@ public class ConsolidateUnnestsRule extends AbstractUsedVariablesProcessingRule
         }
         UnnestOperator unnest2 = (UnnestOperator) op2;
 
-        if (!usedVariables.contains(unnest2.getVariable())) {
-            // Check to see if the unnest2 expression has a scalar implementation.
-            ILogicalExpression logicalExpression2 = (ILogicalExpression) unnest2.getExpressionRef().getValue();
-            if (logicalExpression2.getExpressionTag() != LogicalExpressionTag.FUNCTION_CALL) {
-                return false;
-            }
-            AbstractFunctionCallExpression functionCall2 = (AbstractFunctionCallExpression) logicalExpression2;
-            Function functionInfo2 = (Function) functionCall2.getFunctionInfo();
-            if (!functionInfo2.hasScalarEvaluatorFactory()) {
-                return false;
-            }
-            // Exception for specific path expressions.
-            if (functionCall2.getFunctionIdentifier().equals(BuiltinOperators.DESCENDANT.getFunctionIdentifier())
-                    || functionCall2.getFunctionIdentifier().equals(
-                            BuiltinOperators.DESCENDANT_OR_SELF.getFunctionIdentifier())) {
-                return false;
-            }
-
-            // Find unnest2 variable in unnest1
-            Mutable<ILogicalExpression> unnest1Arg = ExpressionToolbox.findVariableExpression(
-                    unnest1.getExpressionRef(), unnest2.getVariable());
-            if (unnest1Arg == null) {
-                return false;
-            }
+        if (usedVariables.contains(unnest2.getVariable())) {
+            return false;
+        }
 
-            // Replace unnest2 expression in unnest1
-            ScalarFunctionCallExpression child = new ScalarFunctionCallExpression(functionInfo2,
-                    functionCall2.getArguments());
-            unnest1Arg.setValue(child);
+        // Check to see if the unnest2 expression has a scalar implementation.
+        ILogicalExpression logicalExpression2 = (ILogicalExpression) unnest2.getExpressionRef().getValue();
+        if (logicalExpression2.getExpressionTag() != LogicalExpressionTag.FUNCTION_CALL) {
+            return false;
+        }
+        AbstractFunctionCallExpression functionCall2 = (AbstractFunctionCallExpression) logicalExpression2;
+        Function functionInfo2 = (Function) functionCall2.getFunctionInfo();
+        if (!functionInfo2.hasScalarEvaluatorFactory()) {
+            return false;
+        }
+        // Exception for specific path expressions.
+        if (functionCall2.getFunctionIdentifier().equals(BuiltinOperators.DESCENDANT.getFunctionIdentifier())
+                || functionCall2.getFunctionIdentifier().equals(
+                        BuiltinOperators.DESCENDANT_OR_SELF.getFunctionIdentifier())) {
+            return false;
+        }
 
-            // Move input for unnest2 into unnest1
-            unnest1.getInputs().clear();
-            unnest1.getInputs().addAll(unnest2.getInputs());
-            return true;
+        // Find unnest2 variable in unnest1
+        Mutable<ILogicalExpression> unnest1Arg = ExpressionToolbox.findVariableExpression(unnest1.getExpressionRef(),
+                unnest2.getVariable());
+        if (unnest1Arg == null) {
+            return false;
         }
-        return false;
+
+        // Replace unnest2 expression in unnest1
+        ScalarFunctionCallExpression child = new ScalarFunctionCallExpression(functionInfo2,
+                functionCall2.getArguments());
+        unnest1Arg.setValue(child);
+
+        // Move input for unnest2 into unnest1
+        unnest1.getInputs().clear();
+        unnest1.getInputs().addAll(unnest2.getInputs());
+        return true;
     }
 }

http://git-wip-us.apache.org/repos/asf/vxquery/blob/7fc0d93b/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/rules/ConvertAssignToUnnestRule.java
----------------------------------------------------------------------
diff --git a/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/rules/ConvertAssignToUnnestRule.java b/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/rules/ConvertAssignToUnnestRule.java
index 72f6d03..28e54e4 100644
--- a/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/rules/ConvertAssignToUnnestRule.java
+++ b/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/rules/ConvertAssignToUnnestRule.java
@@ -31,12 +31,11 @@ import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.UnnestingFunction
 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.UnnestOperator;
-import edu.uci.ics.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
 
 /**
  * The rule searches for unnest(iterate) operator followed by an assign(child)
  * operator and merges the assign into the unnest operator.
- * 
+ *
  * <pre>
  * Before
  * 
@@ -44,22 +43,23 @@ import edu.uci.ics.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
  *   UNNEST( $v2 : iterate( $v1 ) )
  *   ASSIGN( $v1 : sf1( $v0 ) )
  *   plan__child
- *   
+ * 
  *   where plan__parent does not use $v1 and $v0 is defined in plan__child.
  *   sf1 is a scalar function that has a unnesting implementation.
- *   
+ * 
  * After
  * 
  *   plan__parent
  *   UNNEST( $v2 : uf1( $v0 ) )
  *   plan__child
  * </pre>
- * 
+ *
  * @author prestonc
  */
-public class ConvertAssignToUnnestRule implements IAlgebraicRewriteRule {
+public class ConvertAssignToUnnestRule extends AbstractUsedVariablesProcessingRule {
     @Override
-    public boolean rewritePre(Mutable<ILogicalOperator> opRef, IOptimizationContext context) throws AlgebricksException {
+    protected boolean processOperator(Mutable<ILogicalOperator> opRef, IOptimizationContext context)
+            throws AlgebricksException {
         AbstractLogicalOperator op = (AbstractLogicalOperator) opRef.getValue();
         if (op.getOperatorTag() != LogicalOperatorTag.UNNEST) {
             return false;
@@ -82,6 +82,10 @@ public class ConvertAssignToUnnestRule implements IAlgebraicRewriteRule {
         }
         AssignOperator assign = (AssignOperator) op2;
 
+        if (usedVariables.contains(assign.getVariables())) {
+            return false;
+        }
+
         // Check to see if the expression has an unnesting implementation.
         ILogicalExpression logicalExpression2 = (ILogicalExpression) assign.getExpressions().get(0).getValue();
         if (logicalExpression2.getExpressionTag() != LogicalExpressionTag.FUNCTION_CALL) {
@@ -92,21 +96,16 @@ public class ConvertAssignToUnnestRule implements IAlgebraicRewriteRule {
         if (!functionInfo2.hasUnnestingEvaluatorFactory()) {
             return false;
         }
-        
-        // TODO add checks for variables used that have now been removed.
 
         // Update the unnest parameters.
         unnest.getInputs().clear();
         unnest.getInputs().addAll(assign.getInputs());
-    
-        UnnestingFunctionCallExpression child = new UnnestingFunctionCallExpression(functionInfo2, functionCall2.getArguments());
+
+        UnnestingFunctionCallExpression child = new UnnestingFunctionCallExpression(functionInfo2,
+                functionCall2.getArguments());
         unnest.getExpressionRef().setValue(child);
-        
+
         return true;
     }
 
-    @Override
-    public boolean rewritePost(Mutable<ILogicalOperator> opRef, IOptimizationContext context) {
-        return false;
-    }
 }

http://git-wip-us.apache.org/repos/asf/vxquery/blob/7fc0d93b/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/rules/ConvertFromAlgebricksExpressionsRule.java
----------------------------------------------------------------------
diff --git a/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/rules/ConvertFromAlgebricksExpressionsRule.java b/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/rules/ConvertFromAlgebricksExpressionsRule.java
index f77b93b..b363c85 100644
--- a/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/rules/ConvertFromAlgebricksExpressionsRule.java
+++ b/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/rules/ConvertFromAlgebricksExpressionsRule.java
@@ -45,26 +45,26 @@ import edu.uci.ics.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
  * The rule searches for where the Algebricks builtin function are temporarly in the plan in place of XQuery function.
  * The combination the Algebricks builtin function are replace with boolean XQuery function and the XQuery equivalent
  * function.
- * 
+ *
  * <pre>
  * Before
  * 
  *   plan__parent
  *   %OPERATOR( $v1 : algebricks_function( \@input_expression ) )
  *   plan__child
- *   
+ * 
  *   where the function annotation contains a hint on which xquery expression is represented by the algebricks function.
- *   
- * After 
+ * 
+ * After
  * 
  *   plan__parent
  *   %OPERATOR( $v1 :xquery_expression( \@input_expression ) ) )
  *   plan__child
- *   
- *   note the xquery_expression may include the boolean function to ensure only a true or false result.
  * 
+ *   note the xquery_expression may include the boolean function to ensure only a true or false result.
+ *
  * </pre>
- * 
+ *
  * @author prestonc, shivanim
  */
 public class ConvertFromAlgebricksExpressionsRule implements IAlgebraicRewriteRule {
@@ -77,20 +77,14 @@ public class ConvertFromAlgebricksExpressionsRule implements IAlgebraicRewriteRu
     @SuppressWarnings({ "unchecked", "rawtypes" })
     public ConvertFromAlgebricksExpressionsRule() {
         ALGEBRICKS_MAP.put(AlgebricksBuiltinFunctions.AND, new Pair(BuiltinOperators.AND, null));
-        ALGEBRICKS_MAP.put(AlgebricksBuiltinFunctions.EQ, new Pair(BuiltinOperators.VALUE_EQ,
-                BuiltinOperators.GENERAL_EQ));
-        ALGEBRICKS_MAP.put(AlgebricksBuiltinFunctions.GE, new Pair(BuiltinOperators.VALUE_GE,
-                BuiltinOperators.GENERAL_GE));
-        ALGEBRICKS_MAP.put(AlgebricksBuiltinFunctions.GT, new Pair(BuiltinOperators.VALUE_GT,
-                BuiltinOperators.GENERAL_GT));
+        ALGEBRICKS_MAP.put(AlgebricksBuiltinFunctions.EQ, new Pair(BuiltinOperators.VALUE_EQ, null));
+        ALGEBRICKS_MAP.put(AlgebricksBuiltinFunctions.GE, new Pair(BuiltinOperators.VALUE_GE, null));
+        ALGEBRICKS_MAP.put(AlgebricksBuiltinFunctions.GT, new Pair(BuiltinOperators.VALUE_GT, null));
         ALGEBRICKS_MAP.put(AlgebricksBuiltinFunctions.IS_NULL, new Pair(null, BuiltinFunctions.FN_EMPTY_1));
-        ALGEBRICKS_MAP.put(AlgebricksBuiltinFunctions.LE, new Pair(BuiltinOperators.VALUE_LE,
-                BuiltinOperators.GENERAL_LE));
-        ALGEBRICKS_MAP.put(AlgebricksBuiltinFunctions.LT, new Pair(BuiltinOperators.VALUE_LT,
-                BuiltinOperators.GENERAL_LT));
+        ALGEBRICKS_MAP.put(AlgebricksBuiltinFunctions.LE, new Pair(BuiltinOperators.VALUE_LE, null));
+        ALGEBRICKS_MAP.put(AlgebricksBuiltinFunctions.LT, new Pair(BuiltinOperators.VALUE_LT, null));
         ALGEBRICKS_MAP.put(AlgebricksBuiltinFunctions.NOT, new Pair(null, BuiltinFunctions.FN_NOT_1));
-        ALGEBRICKS_MAP.put(AlgebricksBuiltinFunctions.NEQ, new Pair(BuiltinOperators.VALUE_NE,
-                BuiltinOperators.GENERAL_NE));
+        ALGEBRICKS_MAP.put(AlgebricksBuiltinFunctions.NEQ, new Pair(BuiltinOperators.VALUE_NE, null));
         ALGEBRICKS_MAP.put(AlgebricksBuiltinFunctions.OR, new Pair(BuiltinOperators.OR, null));
     }
 
@@ -109,6 +103,7 @@ public class ConvertFromAlgebricksExpressionsRule implements IAlgebraicRewriteRu
                 modified = true;
             }
         }
+        context.computeAndSetTypeEnvironmentForOperator(opRef.getValue());
         return modified;
     }
 

http://git-wip-us.apache.org/repos/asf/vxquery/blob/7fc0d93b/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/rules/ConvertToAlgebricksExpressionsRule.java
----------------------------------------------------------------------
diff --git a/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/rules/ConvertToAlgebricksExpressionsRule.java b/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/rules/ConvertToAlgebricksExpressionsRule.java
index f1747fc..394c622 100644
--- a/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/rules/ConvertToAlgebricksExpressionsRule.java
+++ b/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/rules/ConvertToAlgebricksExpressionsRule.java
@@ -42,7 +42,7 @@ import edu.uci.ics.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
  * The rule searches for where the XQuery function are used in place of Algebricks builtin function.
  * The combination of the boolean XQuery function and the XQuery equivalent function are replaced with
  * the Algebricks builtin function .
- * 
+ *
  * <pre>
  * Before
  * 
@@ -50,18 +50,18 @@ import edu.uci.ics.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
  *   %OPERATOR( $v1 : xquery_expression( \@input_expression ) )
  *   plan__child
  * 
- *   where xquery_expression has a known equivalent in Algebricks, 
+ *   where xquery_expression has a known equivalent in Algebricks,
  *     such as conditional expressions and a check for null.
  *     The expression may include the boolean function to ensure only
  *     a true or false result.
- *   
- * After 
+ * 
+ * After
  * 
  *   plan__parent
  *   %OPERATOR( $v1 : algebricks_function( \@input_expression ) )
  *   plan__child
  * </pre>
- * 
+ *
  * @author prestonc, shivanim
  */
 public class ConvertToAlgebricksExpressionsRule implements IAlgebraicRewriteRule {
@@ -81,12 +81,6 @@ public class ConvertToAlgebricksExpressionsRule implements IAlgebraicRewriteRule
 
         ALGEBRICKS_MAP.put(BuiltinFunctions.FN_EMPTY_1.getFunctionIdentifier(), AlgebricksBuiltinFunctions.IS_NULL);
         ALGEBRICKS_MAP.put(BuiltinFunctions.FN_NOT_1.getFunctionIdentifier(), AlgebricksBuiltinFunctions.NOT);
-        ALGEBRICKS_MAP.put(BuiltinOperators.GENERAL_EQ.getFunctionIdentifier(), AlgebricksBuiltinFunctions.EQ);
-        ALGEBRICKS_MAP.put(BuiltinOperators.GENERAL_GE.getFunctionIdentifier(), AlgebricksBuiltinFunctions.GE);
-        ALGEBRICKS_MAP.put(BuiltinOperators.GENERAL_GT.getFunctionIdentifier(), AlgebricksBuiltinFunctions.GT);
-        ALGEBRICKS_MAP.put(BuiltinOperators.GENERAL_LE.getFunctionIdentifier(), AlgebricksBuiltinFunctions.LE);
-        ALGEBRICKS_MAP.put(BuiltinOperators.GENERAL_LT.getFunctionIdentifier(), AlgebricksBuiltinFunctions.LT);
-        ALGEBRICKS_MAP.put(BuiltinOperators.GENERAL_NE.getFunctionIdentifier(), AlgebricksBuiltinFunctions.NEQ);
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/vxquery/blob/7fc0d93b/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/rules/InlineNestedVariablesRule.java
----------------------------------------------------------------------
diff --git a/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/rules/InlineNestedVariablesRule.java b/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/rules/InlineNestedVariablesRule.java
deleted file mode 100644
index c0c082f..0000000
--- a/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/rules/InlineNestedVariablesRule.java
+++ /dev/null
@@ -1,97 +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 java.util.List;
-
-import org.apache.commons.lang3.mutable.Mutable;
-
-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.ILogicalPlan;
-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.base.LogicalVariable;
-import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractOperatorWithNestedPlans;
-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AssignOperator;
-import edu.uci.ics.hyracks.algebricks.rewriter.rules.InlineVariablesRule;
-
-/**
- * Modifies the InlineVariablesRule to also process nested plans.
- */
-public class InlineNestedVariablesRule extends InlineVariablesRule {
-
-    protected boolean inlineVariables(Mutable<ILogicalOperator> opRef, IOptimizationContext context)
-            throws AlgebricksException {
-        AbstractLogicalOperator op = (AbstractLogicalOperator) opRef.getValue();
-
-        // Update mapping from variables to expressions during top-down traversal.
-        if (op.getOperatorTag() == LogicalOperatorTag.ASSIGN) {
-            AssignOperator assignOp = (AssignOperator) op;
-            List<LogicalVariable> vars = assignOp.getVariables();
-            List<Mutable<ILogicalExpression>> exprs = assignOp.getExpressions();
-            for (int i = 0; i < vars.size(); i++) {
-                ILogicalExpression expr = exprs.get(i).getValue();
-                // Ignore functions that are either in the doNotInline set or are non-functional               
-                if (expr.getExpressionTag() == LogicalExpressionTag.FUNCTION_CALL) {
-                    AbstractFunctionCallExpression funcExpr = (AbstractFunctionCallExpression) expr;
-                    if (doNotInlineFuncs.contains(funcExpr.getFunctionIdentifier()) || !funcExpr.isFunctional()) {
-                        continue;
-                    }
-                }
-                varAssignRhs.put(vars.get(i), exprs.get(i).getValue());
-            }
-        }
-
-        boolean modified = false;
-        // Descend into nested plans inlining along the way.
-        if (op.hasNestedPlans()) {
-            AbstractOperatorWithNestedPlans nestedOp = (AbstractOperatorWithNestedPlans) op;
-            for (ILogicalPlan nestedPlan : nestedOp.getNestedPlans()) {
-                for (Mutable<ILogicalOperator> nestedOpRef : nestedPlan.getRoots()) {
-                    if (inlineVariables(nestedOpRef, context)) {
-                        modified = true;
-                    }
-                }
-            }
-        }
-
-        // Descend into children inlining along on the way.
-        for (Mutable<ILogicalOperator> inputOpRef : op.getInputs()) {
-            if (inlineVariables(inputOpRef, context)) {
-                modified = true;
-            }
-        }
-
-        if (performBottomUpAction(op)) {
-            modified = true;
-        }
-
-        if (modified) {
-            context.computeAndSetTypeEnvironmentForOperator(op);
-            context.addToDontApplySet(this, op);
-            // Re-enable rules that we may have already tried. They could be applicable now after inlining.
-            context.removeFromAlreadyCompared(opRef.getValue());
-        }
-
-        return modified;
-    }
-}

http://git-wip-us.apache.org/repos/asf/vxquery/blob/7fc0d93b/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/rules/algebricksalternatives/ExtractFunctionsFromJoinConditionRule.java
----------------------------------------------------------------------
diff --git a/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/rules/algebricksalternatives/ExtractFunctionsFromJoinConditionRule.java b/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/rules/algebricksalternatives/ExtractFunctionsFromJoinConditionRule.java
new file mode 100644
index 0000000..9847c7f
--- /dev/null
+++ b/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/rules/algebricksalternatives/ExtractFunctionsFromJoinConditionRule.java
@@ -0,0 +1,141 @@
+/*
+ * 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.algebricksalternatives;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.commons.lang3.mutable.Mutable;
+import org.apache.commons.lang3.mutable.MutableObject;
+
+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.base.LogicalVariable;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.VariableReferenceExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.functions.AlgebricksBuiltinFunctions;
+import edu.uci.ics.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+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.AssignOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.visitors.VariableUtilities;
+import edu.uci.ics.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+
+/**
+ * Factors out function expressions from each comparison function or similarity function in join condition by assigning them to a variables, and replacing the function expressions with references to those variables.
+ * Examples:
+ * Plan with function expressions in comparison or similarity condition of join expression. Generates one assign operator per extracted function expression.
+ * Example
+ * Before plan:
+ * join ( eq( funcX($$1), funcX($$2) ) )
+ * After plan:
+ * join (eq($$3,$$4))
+ * assign [$$4] <- [funcY($$2)]
+ * assign [$$3] <- [funcX($$1)]
+ */
+public class ExtractFunctionsFromJoinConditionRule implements IAlgebraicRewriteRule {
+
+    @Override
+    public boolean rewritePre(Mutable<ILogicalOperator> opRef, IOptimizationContext context) throws AlgebricksException {
+        return false;
+    }
+
+    @Override
+    public boolean rewritePost(Mutable<ILogicalOperator> opRef, IOptimizationContext context)
+            throws AlgebricksException {
+        AbstractLogicalOperator op = (AbstractLogicalOperator) opRef.getValue();
+
+        if (op.getOperatorTag() != LogicalOperatorTag.INNERJOIN
+                && op.getOperatorTag() != LogicalOperatorTag.LEFTOUTERJOIN) {
+            return false;
+        }
+        AbstractBinaryJoinOperator joinOp = (AbstractBinaryJoinOperator) op;
+        ILogicalExpression expr = joinOp.getCondition().getValue();
+
+        return assignFunctionExpressions(joinOp, expr, context);
+
+    }
+
+    private boolean assignFunctionExpressions(AbstractLogicalOperator joinOp, ILogicalExpression expr,
+            IOptimizationContext context) throws AlgebricksException {
+        if (expr.getExpressionTag() != LogicalExpressionTag.FUNCTION_CALL) {
+            return false;
+        }
+        AbstractFunctionCallExpression fexp = (AbstractFunctionCallExpression) expr;
+        FunctionIdentifier fi = fexp.getFunctionIdentifier();
+
+        boolean modified = false;
+        if (fi.equals(AlgebricksBuiltinFunctions.AND) || fi.equals(AlgebricksBuiltinFunctions.OR)) {
+            for (Mutable<ILogicalExpression> a : fexp.getArguments()) {
+                if (assignFunctionExpressions(joinOp, a.getValue(), context)) {
+                    modified = true;
+                }
+            }
+            return modified;
+        } else if (AlgebricksBuiltinFunctions.isComparisonFunction(fi)) {
+            for (Mutable<ILogicalExpression> exprRef : fexp.getArguments()) {
+                if (exprRef.getValue().getExpressionTag() == LogicalExpressionTag.FUNCTION_CALL) {
+                    LogicalVariable newVar = context.newVar();
+                    AssignOperator newAssign = new AssignOperator(newVar, new MutableObject<ILogicalExpression>(exprRef
+                            .getValue().cloneExpression()));
+                    newAssign.setExecutionMode(joinOp.getExecutionMode());
+
+                    // Place assign below joinOp.
+                    List<LogicalVariable> used = new ArrayList<LogicalVariable>();
+                    VariableUtilities.getUsedVariables(newAssign, used);
+
+                    Mutable<ILogicalOperator> leftBranchRef = joinOp.getInputs().get(0);
+                    ILogicalOperator leftBranch = leftBranchRef.getValue();
+                    List<LogicalVariable> leftBranchVariables = new ArrayList<LogicalVariable>();
+                    VariableUtilities.getLiveVariables(leftBranch, leftBranchVariables);
+                    if (leftBranchVariables.containsAll(used)) {
+                        // place assign on left branch
+                        newAssign.getInputs().add(new MutableObject<ILogicalOperator>(leftBranch));
+                        leftBranchRef.setValue(newAssign);
+                        modified = true;
+                    } else {
+                        Mutable<ILogicalOperator> rightBranchRef = joinOp.getInputs().get(1);
+                        ILogicalOperator rightBranch = rightBranchRef.getValue();
+                        List<LogicalVariable> rightBranchVariables = new ArrayList<LogicalVariable>();
+                        VariableUtilities.getLiveVariables(rightBranch, rightBranchVariables);
+                        if (rightBranchVariables.containsAll(used)) {
+                            // place assign on right branch
+                            newAssign.getInputs().add(new MutableObject<ILogicalOperator>(rightBranch));
+                            rightBranchRef.setValue(newAssign);
+                            modified = true;
+                        }
+                    }
+
+                    if (modified) {
+                        // Replace original expr with variable reference.
+                        exprRef.setValue(new VariableReferenceExpression(newVar));
+                        context.computeAndSetTypeEnvironmentForOperator(newAssign);
+                        context.computeAndSetTypeEnvironmentForOperator(joinOp);
+                    }
+                }
+            }
+            return modified;
+        } else {
+            return false;
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/vxquery/blob/7fc0d93b/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/rules/algebricksalternatives/InlineNestedVariablesRule.java
----------------------------------------------------------------------
diff --git a/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/rules/algebricksalternatives/InlineNestedVariablesRule.java b/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/rules/algebricksalternatives/InlineNestedVariablesRule.java
new file mode 100644
index 0000000..ca0bc1c
--- /dev/null
+++ b/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/rules/algebricksalternatives/InlineNestedVariablesRule.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.algebricksalternatives;
+
+import java.util.List;
+
+import org.apache.commons.lang3.mutable.Mutable;
+
+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.ILogicalPlan;
+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.base.LogicalVariable;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractOperatorWithNestedPlans;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AssignOperator;
+import edu.uci.ics.hyracks.algebricks.rewriter.rules.InlineVariablesRule;
+
+/**
+ * Modifies the InlineVariablesRule to also process nested plans.
+ */
+public class InlineNestedVariablesRule extends InlineVariablesRule {
+
+    protected boolean inlineVariables(Mutable<ILogicalOperator> opRef, IOptimizationContext context)
+            throws AlgebricksException {
+        AbstractLogicalOperator op = (AbstractLogicalOperator) opRef.getValue();
+
+        // Update mapping from variables to expressions during top-down traversal.
+        if (op.getOperatorTag() == LogicalOperatorTag.ASSIGN) {
+            AssignOperator assignOp = (AssignOperator) op;
+            List<LogicalVariable> vars = assignOp.getVariables();
+            List<Mutable<ILogicalExpression>> exprs = assignOp.getExpressions();
+            for (int i = 0; i < vars.size(); i++) {
+                ILogicalExpression expr = exprs.get(i).getValue();
+                // Ignore functions that are either in the doNotInline set or are non-functional
+                if (expr.getExpressionTag() == LogicalExpressionTag.FUNCTION_CALL) {
+                    AbstractFunctionCallExpression funcExpr = (AbstractFunctionCallExpression) expr;
+                    if (doNotInlineFuncs.contains(funcExpr.getFunctionIdentifier()) || !funcExpr.isFunctional()) {
+                        continue;
+                    }
+                }
+                varAssignRhs.put(vars.get(i), exprs.get(i).getValue());
+            }
+        }
+
+        boolean modified = false;
+        // Descend into nested plans inlining along the way.
+        if (op.hasNestedPlans()) {
+            AbstractOperatorWithNestedPlans nestedOp = (AbstractOperatorWithNestedPlans) op;
+            for (ILogicalPlan nestedPlan : nestedOp.getNestedPlans()) {
+                for (Mutable<ILogicalOperator> nestedOpRef : nestedPlan.getRoots()) {
+                    if (inlineVariables(nestedOpRef, context)) {
+                        modified = true;
+                    }
+                }
+            }
+        }
+
+        // Descend into children inlining along on the way.
+        for (Mutable<ILogicalOperator> inputOpRef : op.getInputs()) {
+            if (inlineVariables(inputOpRef, context)) {
+                modified = true;
+            }
+        }
+
+        if (performBottomUpAction(op)) {
+            modified = true;
+        }
+
+        if (modified) {
+            context.computeAndSetTypeEnvironmentForOperator(op);
+            context.addToDontApplySet(this, op);
+            // Re-enable rules that we may have already tried. They could be applicable now after inlining.
+            context.removeFromAlreadyCompared(opRef.getValue());
+        }
+
+        return modified;
+    }
+}

http://git-wip-us.apache.org/repos/asf/vxquery/blob/7fc0d93b/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/rules/algebricksalternatives/MoveFreeVariableOperatorOutOfSubplanRule.java
----------------------------------------------------------------------
diff --git a/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/rules/algebricksalternatives/MoveFreeVariableOperatorOutOfSubplanRule.java b/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/rules/algebricksalternatives/MoveFreeVariableOperatorOutOfSubplanRule.java
new file mode 100644
index 0000000..94b809d
--- /dev/null
+++ b/vxquery-core/src/main/java/org/apache/vxquery/compiler/rewriter/rules/algebricksalternatives/MoveFreeVariableOperatorOutOfSubplanRule.java
@@ -0,0 +1,178 @@
+/*
+ * 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.algebricksalternatives;
+
+import java.util.HashSet;
+import java.util.ListIterator;
+import java.util.Set;
+
+import org.apache.commons.lang3.mutable.Mutable;
+
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalPlan;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+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.AbstractLogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.SubplanOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.visitors.VariableUtilities;
+import edu.uci.ics.hyracks.algebricks.core.algebra.util.OperatorPropertiesUtil;
+import edu.uci.ics.hyracks.algebricks.rewriter.rules.AbstractDecorrelationRule;
+
+/**
+ * The rule searches for operators that can be moved outside the subplan.
+ *
+ * <pre>
+ * Before
+ * 
+ *   %PARENT_PLAN
+ *   SUBPLAN{
+ *     %NESTED_OPERATORS_B+
+ *     ASSIGN || %SUBPLAN
+ *     %NESTED_OPERATORS_A*
+ *     NESTEDTUPLESOURCE
+ *   }
+ *   %CHILD_PLAN
+ * 
+ *   where
+ *     %SUBPLAN has one nested plan with a root AGGREGATE operator.
+ * 
+ * After
+ * 
+ *   %PARENT_PLAN
+ *   SUBPLAN{
+ *     %NESTED_OPERATORS_B+
+ *     %NESTED_OPERATORS_A*
+ *     NESTEDTUPLESOURCE
+ *   }
+ *   ASSIGN || %SUBPLAN
+ *   %CHILD_PLAN
+ * </pre>
+ */
+public class MoveFreeVariableOperatorOutOfSubplanRule extends AbstractDecorrelationRule {
+
+    @Override
+    public boolean rewritePost(Mutable<ILogicalOperator> opRef, IOptimizationContext context)
+            throws AlgebricksException {
+        AbstractLogicalOperator op0 = (AbstractLogicalOperator) opRef.getValue();
+        if (op0.getOperatorTag() != LogicalOperatorTag.SUBPLAN) {
+            return false;
+        }
+        SubplanOperator subplan = (SubplanOperator) op0;
+
+        Mutable<ILogicalOperator> leftRef = subplan.getInputs().get(0);
+        if (((AbstractLogicalOperator) leftRef.getValue()).getOperatorTag() == LogicalOperatorTag.EMPTYTUPLESOURCE) {
+            return false;
+        }
+
+        ListIterator<ILogicalPlan> plansIter = subplan.getNestedPlans().listIterator();
+        ILogicalPlan p = null;
+        while (plansIter.hasNext()) {
+            p = plansIter.next();
+        }
+        if (p == null) {
+            return false;
+        }
+        if (p.getRoots().size() != 1) {
+            return false;
+        }
+        Mutable<ILogicalOperator> opRef1 = p.getRoots().get(0);
+
+        //The root operator will not be movable. Start with the second op
+        AbstractLogicalOperator op1 = (AbstractLogicalOperator) opRef1.getValue();
+        if (op1.getInputs().size() != 1) {
+            return false;
+        }
+        Mutable<ILogicalOperator> op2Ref = op1.getInputs().get(0);
+
+        //Get all variables that come from outside of the loop
+        Set<LogicalVariable> free = new HashSet<LogicalVariable>();
+        OperatorPropertiesUtil.getFreeVariablesInSelfOrDesc(op1, free);
+
+        while (op2Ref != null) {
+            //Get the operator that we want to look at
+            AbstractLogicalOperator op2 = (AbstractLogicalOperator) op2Ref.getValue();
+
+            //Make sure we are looking at subplan with a scan/join
+            if (op2.getInputs().size() != 1 || !descOrSelfIsScanOrJoin(op2)) {
+                return false;
+            }
+            boolean notApplicable = false;
+
+            //Get its used variables
+            Set<LogicalVariable> used = new HashSet<LogicalVariable>();
+
+            //not movable if the operator is not an assign or subplan
+            //Might be helpful in the future for other operations in the future
+            if (op2.getOperatorTag() == LogicalOperatorTag.ASSIGN) {
+                VariableUtilities.getUsedVariables(op2, used);
+            } else if (op2.getOperatorTag() == LogicalOperatorTag.SUBPLAN) {
+                // Nested plan must have an aggregate root.
+                ListIterator<ILogicalPlan> subplansIter = ((SubplanOperator) op2).getNestedPlans().listIterator();
+                ILogicalPlan plan = null;
+                while (subplansIter.hasNext()) {
+                    plan = subplansIter.next();
+                }
+                if (plan == null) {
+                    return false;
+                }
+                if (plan.getRoots().size() != 1) {
+                    return false;
+                }
+                ILogicalOperator op3 = plan.getRoots().get(0).getValue();
+                if (op3.getOperatorTag() != LogicalOperatorTag.AGGREGATE) {
+                    return false;
+                }
+                // Used variables do not include ones created in the subplan.
+                VariableUtilities.getUsedVariables(op2, used);
+                Set<LogicalVariable> subplanProducedAndDown = new HashSet<LogicalVariable>();
+                VariableUtilities.getProducedVariablesInDescendantsAndSelf(op3, subplanProducedAndDown);
+                used.removeAll(subplanProducedAndDown);
+            } else {
+                notApplicable = true;
+            }
+
+            //Make sure that all of its used variables come from outside
+            for (LogicalVariable var : used) {
+                if (!free.contains(var)) {
+                    notApplicable = true;
+                }
+            }
+
+            if (notApplicable) {
+                op2Ref = op2.getInputs().get(0);
+            } else {
+                //Make the input of op2 be the input of op1
+                op2Ref.setValue(op2.getInputs().get(0).getValue());
+
+                //Make the outside of the subplan the input of op2
+                Mutable<ILogicalOperator> outsideRef = op2.getInputs().get(0);
+                outsideRef.setValue(op0.getInputs().get(0).getValue());
+
+                //Make op2 the input of the subplan
+                Mutable<ILogicalOperator> op2OutsideRef = op0.getInputs().get(0);
+                op2OutsideRef.setValue(op2);
+
+                return true;
+            }
+
+        }
+        return false;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/vxquery/blob/7fc0d93b/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/comparison/general/AbstractGeneralComparisonScalarEvaluatorFactory.java
----------------------------------------------------------------------
diff --git a/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/comparison/general/AbstractGeneralComparisonScalarEvaluatorFactory.java b/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/comparison/general/AbstractGeneralComparisonScalarEvaluatorFactory.java
index 446b445..99c4faa 100644
--- a/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/comparison/general/AbstractGeneralComparisonScalarEvaluatorFactory.java
+++ b/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/comparison/general/AbstractGeneralComparisonScalarEvaluatorFactory.java
@@ -146,7 +146,7 @@ public abstract class AbstractGeneralComparisonScalarEvaluatorFactory extends
 
             /**
              * Check the second argument for a sequence and loop if required.
-             * 
+             *
              * @param aOp
              * @param tvpArg1
              * @param tvpArg2
@@ -181,7 +181,7 @@ public abstract class AbstractGeneralComparisonScalarEvaluatorFactory extends
 
             /**
              * Transform the values into values supported for general comparison.
-             * 
+             *
              * @param aOp
              * @param tvpArg1
              * @param tvpArg2
@@ -219,8 +219,16 @@ public abstract class AbstractGeneralComparisonScalarEvaluatorFactory extends
                     // Set up value comparison tagged value pointables.
                     if (tid1 == ValueTag.XS_UNTYPED_ATOMIC_TAG && tid2 == ValueTag.XS_UNTYPED_ATOMIC_TAG) {
                         // Only need to change tag since the storage is the same for untyped atomic and string.
+                        dOutInner1.write(tvpTransform1.getByteArray(), tvpTransform1.getStartOffset(),
+                                tvpTransform1.getLength());
+                        tvpCompare1.set(abvsInner1.getByteArray(), abvsInner1.getStartOffset(), abvsInner1.getLength());
                         tvpCompare1.getByteArray()[0] = ValueTag.XS_STRING_TAG;
+                        tagTransformed1 = true;
+                        dOutInner2.write(tvpTransform2.getByteArray(), tvpTransform2.getStartOffset(),
+                                tvpTransform2.getLength());
+                        tvpCompare2.set(abvsInner2.getByteArray(), abvsInner2.getStartOffset(), abvsInner2.getLength());
                         tvpCompare2.getByteArray()[0] = ValueTag.XS_STRING_TAG;
+                        tagTransformed2 = true;
                     } else if (tid1 == ValueTag.XS_UNTYPED_ATOMIC_TAG) {
                         tid1 = tid2;
                         getCastToOperator(tid2);

http://git-wip-us.apache.org/repos/asf/vxquery/blob/7fc0d93b/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/conditional/IfThenElseScalarEvaluatorFactory.java
----------------------------------------------------------------------
diff --git a/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/conditional/IfThenElseScalarEvaluatorFactory.java b/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/conditional/IfThenElseScalarEvaluatorFactory.java
index b0584f8..7cd97e3 100644
--- a/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/conditional/IfThenElseScalarEvaluatorFactory.java
+++ b/vxquery-core/src/main/java/org/apache/vxquery/runtime/functions/conditional/IfThenElseScalarEvaluatorFactory.java
@@ -20,7 +20,6 @@ import org.apache.vxquery.datamodel.accessors.TaggedValuePointable;
 import org.apache.vxquery.datamodel.values.ValueTag;
 import org.apache.vxquery.exceptions.ErrorCode;
 import org.apache.vxquery.exceptions.SystemException;
-import org.apache.vxquery.runtime.functions.base.AbstractTaggedValueArgumentScalarEvaluator;
 import org.apache.vxquery.runtime.functions.base.AbstractTaggedValueArgumentScalarEvaluatorFactory;
 
 import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
@@ -29,32 +28,42 @@ import edu.uci.ics.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
 import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.data.std.api.IPointable;
 import edu.uci.ics.hyracks.data.std.primitive.BooleanPointable;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
 
 public class IfThenElseScalarEvaluatorFactory extends AbstractTaggedValueArgumentScalarEvaluatorFactory {
     private static final long serialVersionUID = 1L;
+    private static final int CONDITION = 0;
+    private static final int TRUE_CONDITION = 1;
+    private static final int FALSE_CONDITION = 2;
 
     public IfThenElseScalarEvaluatorFactory(IScalarEvaluatorFactory[] args) {
         super(args);
     }
 
     @Override
-    protected IScalarEvaluator createEvaluator(IHyracksTaskContext ctx, IScalarEvaluator[] args)
+    protected IScalarEvaluator createEvaluator(IHyracksTaskContext ctx, final IScalarEvaluator[] args)
             throws AlgebricksException {
-        return new AbstractTaggedValueArgumentScalarEvaluator(args) {
-            private final BooleanPointable bp = (BooleanPointable) BooleanPointable.FACTORY.createPointable();
+        final TaggedValuePointable conditionTvp = (TaggedValuePointable) TaggedValuePointable.FACTORY.createPointable();
+        final BooleanPointable bp = (BooleanPointable) BooleanPointable.FACTORY.createPointable();
 
+        return new IScalarEvaluator() {
             @Override
-            protected void evaluate(TaggedValuePointable[] args, IPointable result) throws SystemException {
-                TaggedValuePointable tvp1 = args[0];
-                if (tvp1.getTag() != ValueTag.XS_BOOLEAN_TAG) {
-                    throw new SystemException(ErrorCode.FORG0006);
-                }
-                tvp1.getValue(bp);
+            public void evaluate(IFrameTupleReference tuple, IPointable result) throws AlgebricksException {
+                args[CONDITION].evaluate(tuple, conditionTvp);
+
+                try {
+                    if (conditionTvp.getTag() != ValueTag.XS_BOOLEAN_TAG) {
+                        throw new SystemException(ErrorCode.FORG0006);
+                    }
+                    conditionTvp.getValue(bp);
 
-                if (bp.getBoolean()) {
-                    result.set(args[1]);
-                } else {
-                    result.set(args[2]);
+                    if (bp.getBoolean()) {
+                        args[TRUE_CONDITION].evaluate(tuple, result);
+                    } else {
+                        args[FALSE_CONDITION].evaluate(tuple, result);
+                    }
+                } catch (SystemException e) {
+                    throw new AlgebricksException(e);
                 }
             }
         };

http://git-wip-us.apache.org/repos/asf/vxquery/blob/7fc0d93b/vxquery-core/src/main/java/org/apache/vxquery/serializer/XMLSerializer.java
----------------------------------------------------------------------
diff --git a/vxquery-core/src/main/java/org/apache/vxquery/serializer/XMLSerializer.java b/vxquery-core/src/main/java/org/apache/vxquery/serializer/XMLSerializer.java
index 189ae24..8a5845d 100644
--- a/vxquery-core/src/main/java/org/apache/vxquery/serializer/XMLSerializer.java
+++ b/vxquery-core/src/main/java/org/apache/vxquery/serializer/XMLSerializer.java
@@ -366,7 +366,7 @@ public class XMLSerializer implements IPrinter {
             enp.getAttributeSequence(ntp, seqp);
             if (seqp.getByteArray() != null && seqp.getEntryCount() > 0) {
                 ps.append(' ');
-                printSequence(ps, seqp);
+                printSequence(ps, seqp, " ");
             }
 
             enp.getChildrenSequence(ntp, seqp);
@@ -422,12 +422,19 @@ public class XMLSerializer implements IPrinter {
     }
 
     private void printSequence(PrintStream ps, SequencePointable seqp) {
+        printSequence(ps, seqp, null);
+    }
+
+    private void printSequence(PrintStream ps, SequencePointable seqp, String between) {
         VoidPointable vp = pp.takeOne(VoidPointable.class);
         try {
             int len = seqp.getEntryCount();
             for (int i = 0; i < len; ++i) {
                 seqp.getEntry(i, vp);
                 print(vp.getByteArray(), vp.getStartOffset(), vp.getLength(), ps);
+                if (i < len - 1 && between != null) {
+                    ps.append(between);
+                }
             }
         } finally {
             pp.giveBack(vp);

http://git-wip-us.apache.org/repos/asf/vxquery/blob/7fc0d93b/vxquery-core/src/main/java/org/apache/vxquery/xmlquery/query/XMLQueryCompiler.java
----------------------------------------------------------------------
diff --git a/vxquery-core/src/main/java/org/apache/vxquery/xmlquery/query/XMLQueryCompiler.java b/vxquery-core/src/main/java/org/apache/vxquery/xmlquery/query/XMLQueryCompiler.java
index 38272dd..d9c5d01 100644
--- a/vxquery-core/src/main/java/org/apache/vxquery/xmlquery/query/XMLQueryCompiler.java
+++ b/vxquery-core/src/main/java/org/apache/vxquery/xmlquery/query/XMLQueryCompiler.java
@@ -164,7 +164,13 @@ public class XMLQueryCompiler {
         builder.setNullableTypeComputer(new INullableTypeComputer() {
             @Override
             public Object makeNullableType(Object type) throws AlgebricksException {
-                throw new NotImplementedException("NullableTypeComputer is not implented (makeNullableType)");
+                SequenceType st = (SequenceType) type;
+                if (st.getQuantifier().allowsEmptySequence()) {
+                    return type;
+                } else if (st.getQuantifier() == Quantifier.QUANT_ONE) {
+                    return SequenceType.create(st.getItemType(), Quantifier.QUANT_QUESTION);
+                }
+                return type;
             }
 
             @Override
@@ -239,6 +245,8 @@ public class XMLQueryCompiler {
                 RewriteRuleset.buildNestedDataSourceRuleCollection()));
         defaultLogicalRewrites.add(new Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>(seqOnceCtrl,
                 RewriteRuleset.buildTypeInferenceRuleCollection()));
+        defaultLogicalRewrites.add(new Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>(seqCtrlNoDfs,
+                RewriteRuleset.buildUnnestingRuleCollection()));
         defaultLogicalRewrites.add(new Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>(seqCtrlFullDfs,
                 RewriteRuleset.buildNormalizationRuleCollection()));
         defaultLogicalRewrites.add(new Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>(seqCtrlNoDfs,

http://git-wip-us.apache.org/repos/asf/vxquery/blob/7fc0d93b/vxquery-core/src/main/java/org/apache/vxquery/xmlquery/translator/XMLQueryTranslator.java
----------------------------------------------------------------------
diff --git a/vxquery-core/src/main/java/org/apache/vxquery/xmlquery/translator/XMLQueryTranslator.java b/vxquery-core/src/main/java/org/apache/vxquery/xmlquery/translator/XMLQueryTranslator.java
index 2507d6c..9faefda 100644
--- a/vxquery-core/src/main/java/org/apache/vxquery/xmlquery/translator/XMLQueryTranslator.java
+++ b/vxquery-core/src/main/java/org/apache/vxquery/xmlquery/translator/XMLQueryTranslator.java
@@ -160,7 +160,9 @@ 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.ILogicalPlan;
 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.AggregateFunctionCallExpression;
 import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.ConstantExpression;
 import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.ScalarFunctionCallExpression;
@@ -1501,26 +1503,40 @@ public class XMLQueryTranslator {
                     ctxExpr = vre(createAssignment(ctxExpr, tCtx));
                     ctxExpr = sfce(asc ? BuiltinOperators.SORT_DISTINCT_NODES_ASC_OR_ATOMICS
                             : BuiltinOperators.SORT_DISTINCT_NODES_DESC_OR_ATOMICS, ctxExpr);
+                    iterateOver(ctxExpr, tCtx);
+                    int i = 0;
+                    ILogicalExpression selectCondition = null;
                     for (ASTNode pn : predicates) {
-                        tCtx = tCtx.pushContext();
-                        tCtx.pushVariableScope();
-                        iterateOver(ctxExpr, tCtx);
+                        // Handles integer, boolean expression, path expression exists
+                        // TODO Support inner focus between predicates.
                         LogicalVariable pLVar = translateExpression(pn, tCtx);
                         ILogicalExpression tTest = instanceOf(vre(pLVar),
                                 SequenceType.create(BuiltinTypeRegistry.XSEXT_NUMERIC, Quantifier.QUANT_ONE));
                         ILogicalExpression posTest = sfce(BuiltinOperators.VALUE_EQ, vre(pLVar), vre(tCtx.varScope
                                 .lookupVariable(XMLQueryCompilerConstants.POS_VAR_NAME).getLogicalVariable()));
-                        ILogicalExpression boolTest = sfce(BuiltinFunctions.FN_BOOLEAN_1, vre(pLVar));
-
-                        SelectOperator select = new SelectOperator(mutable(sfce(BuiltinOperators.IF_THEN_ELSE, tTest,
-                                posTest, boolTest)), false, null);
-                        select.getInputs().add(mutable(tCtx.op));
-                        tCtx.op = select;
-                        ctxExpr = vre(tCtx.varScope.lookupVariable(XMLQueryCompilerConstants.DOT_VAR_NAME)
-                                .getLogicalVariable());
-                        tCtx.popVariableScope();
-                        tCtx = tCtx.popContext();
+                        ILogicalExpression boolTestTmp = vre(pLVar);
+                        if (tCtx.op.getOperatorTag() == LogicalOperatorTag.ASSIGN) {
+                            ILogicalExpression expression = ((AssignOperator) tCtx.op).getExpressions().get(0)
+                                    .getValue();
+                            if (expression.getExpressionTag() == LogicalExpressionTag.FUNCTION_CALL
+                                    && ((AbstractFunctionCallExpression) expression).getFunctionIdentifier().equals(
+                                            BuiltinOperators.CHILD.getFunctionIdentifier())) {
+                                boolTestTmp = sfce(BuiltinFunctions.FN_COUNT_1, boolTestTmp);
+                            }
+                        }
+                        ILogicalExpression boolTest = sfce(BuiltinFunctions.FN_BOOLEAN_1, boolTestTmp);
+                        ILogicalExpression condition = sfce(BuiltinOperators.IF_THEN_ELSE, tTest, posTest, boolTest);
+                        if (i++ == 0) {
+                            selectCondition = condition;
+                        } else {
+                            selectCondition = sfce(BuiltinOperators.AND, selectCondition, condition);
+                        }
                     }
+                    SelectOperator select = new SelectOperator(mutable(selectCondition), false, null);
+                    select.getInputs().add(mutable(tCtx.op));
+                    tCtx.op = select;
+                    ctxExpr = vre(tCtx.varScope.lookupVariable(XMLQueryCompilerConstants.DOT_VAR_NAME)
+                            .getLogicalVariable());
                 }
                 if (popScope) {
                     tCtx.popVariableScope();

http://git-wip-us.apache.org/repos/asf/vxquery/blob/7fc0d93b/vxquery-xtest/pom.xml
----------------------------------------------------------------------
diff --git a/vxquery-xtest/pom.xml b/vxquery-xtest/pom.xml
index 6fb2798..9e83fd7 100644
--- a/vxquery-xtest/pom.xml
+++ b/vxquery-xtest/pom.xml
@@ -119,6 +119,7 @@
                      <!-- <argLine>-agentpath:"${yourkit.home}/bin/mac/libyjpagent.jnilib=sampling"</argLine> -->
                     <excludes>
                         <exclude>**/AbstractXQueryTest.java</exclude>
+                        <exclude>**/VXQueryXMarkTest.java</exclude>
                         <exclude>**/XMarkTest.java</exclude>
                         <exclude>**/XQTSTest.java</exclude>
                     </excludes>