Moving materialize operator and nested subplan to join rule to Hyracks.

Change-Id: I5a18d30a9b097ec890c03a0c0f30ebecfae46b3e
Reviewed-on: http://fulliautomatix.ics.uci.edu:8443/169
Tested-by: Jenkins <jenkins@fulliautomatix.ics.uci.edu>
Reviewed-by: Till Westmann <westmann@gmail.com>
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/base/LogicalOperatorDeepCopyVisitor.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/base/LogicalOperatorDeepCopyVisitor.java
index 15b64db..cedc962 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/base/LogicalOperatorDeepCopyVisitor.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/base/LogicalOperatorDeepCopyVisitor.java
@@ -46,6 +46,7 @@
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.InsertDeleteOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.LeftOuterJoinOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.LimitOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.MaterializeOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.NestedTupleSourceOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.OrderOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.OrderOperator.IOrder;
@@ -350,6 +351,12 @@
     }
 
     @Override
+    public ILogicalOperator visitMaterializeOperator(MaterializeOperator op, ILogicalOperator arg)
+            throws AlgebricksException {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
     public ILogicalOperator visitRunningAggregateOperator(RunningAggregateOperator op, ILogicalOperator arg) {
         throw new UnsupportedOperationException();
     }
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/operators/MaterializeOperator.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/operators/MaterializeOperator.java
deleted file mode 100644
index a8e8049..0000000
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/operators/MaterializeOperator.java
+++ /dev/null
@@ -1,57 +0,0 @@
-/*
- * Copyright 2009-2013 by The Regents of the University of California
- * Licensed 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 from
- * 
- *     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 edu.uci.ics.asterix.algebra.operators;
-
-import java.util.Collection;
-
-import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
-import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractExtensibleLogicalOperator;
-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.IOperatorExtension;
-import edu.uci.ics.hyracks.algebricks.core.algebra.visitors.ILogicalExpressionReferenceTransform;
-
-public class MaterializeOperator extends AbstractExtensibleLogicalOperator {
-
-    @Override
-    public boolean isMap() {
-        return false;
-    }
-
-    @Override
-    public IOperatorExtension newInstance() {
-        return new MaterializeOperator();
-    }
-
-    @Override
-    public boolean acceptExpressionTransform(ILogicalExpressionReferenceTransform transform) throws AlgebricksException {
-        return false;
-    }
-
-    @Override
-    public String toString() {
-        return "materialize";
-    }
-
-    @Override
-    public void getUsedVariables(Collection<LogicalVariable> usedVars) {
-        // No used variables.
-    }
-
-    @Override
-    public void getProducedVariables(Collection<LogicalVariable> producedVars) {
-        // No produced variables.
-    }
-}
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/operators/physical/MaterializePOperator.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/operators/physical/MaterializePOperator.java
deleted file mode 100644
index 542e36d..0000000
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/operators/physical/MaterializePOperator.java
+++ /dev/null
@@ -1,100 +0,0 @@
-/*
- * Copyright 2009-2013 by The Regents of the University of California
- * Licensed 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 from
- * 
- *     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 edu.uci.ics.asterix.algebra.operators.physical;
-
-import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
-import edu.uci.ics.hyracks.algebricks.common.utils.Pair;
-import edu.uci.ics.hyracks.algebricks.core.algebra.base.IHyracksJobBuilder;
-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.PhysicalOperatorTag;
-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.IOperatorSchema;
-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical.AbstractPhysicalOperator;
-import edu.uci.ics.hyracks.algebricks.core.algebra.properties.IPhysicalPropertiesVector;
-import edu.uci.ics.hyracks.algebricks.core.algebra.properties.PhysicalRequirements;
-import edu.uci.ics.hyracks.algebricks.core.algebra.properties.StructuralPropertiesVector;
-import edu.uci.ics.hyracks.algebricks.core.jobgen.impl.JobGenContext;
-import edu.uci.ics.hyracks.algebricks.core.jobgen.impl.JobGenHelper;
-import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
-import edu.uci.ics.hyracks.dataflow.std.misc.MaterializingOperatorDescriptor;
-
-public class MaterializePOperator extends AbstractPhysicalOperator {
-
-    private final boolean isSingleActivity;
-
-    public MaterializePOperator(boolean isSingleActivity) {
-        this.isSingleActivity = isSingleActivity;
-    }
-
-    @Override
-    public PhysicalOperatorTag getOperatorTag() {
-        return PhysicalOperatorTag.EXTENSION_OPERATOR;
-    }
-
-    @Override
-    public String toString() {
-        return "MATERIALIZE";
-    }
-
-    @Override
-    public PhysicalRequirements getRequiredPropertiesForChildren(ILogicalOperator op,
-            IPhysicalPropertiesVector reqdByParent) {
-        return emptyUnaryRequirements();
-    }
-
-    @Override
-    public void computeDeliveredProperties(ILogicalOperator op, IOptimizationContext context)
-            throws AlgebricksException {
-        AbstractLogicalOperator op2 = (AbstractLogicalOperator) op.getInputs().get(0).getValue();
-        deliveredProperties = (StructuralPropertiesVector) op2.getDeliveredPhysicalProperties().clone();
-    }
-
-    @Override
-    public void contributeRuntimeOperator(IHyracksJobBuilder builder, JobGenContext context, ILogicalOperator op,
-            IOperatorSchema propagatedSchema, IOperatorSchema[] inputSchemas, IOperatorSchema outerPlanSchema)
-            throws AlgebricksException {
-        RecordDescriptor recDescriptor = JobGenHelper.mkRecordDescriptor(context.getTypeEnvironment(op),
-                propagatedSchema, context);
-        MaterializingOperatorDescriptor materializationOpDesc = new MaterializingOperatorDescriptor(
-                builder.getJobSpec(), recDescriptor, isSingleActivity);
-        contributeOpDesc(builder, (AbstractLogicalOperator) op, materializationOpDesc);
-        ILogicalOperator src = op.getInputs().get(0).getValue();
-        builder.contributeGraphEdge(src, 0, op, 0);
-    }
-
-    @Override
-    public boolean isMicroOperator() {
-        return true;
-    }
-
-    @Override
-    public Pair<int[], int[]> getInputOutputDependencyLabels(ILogicalOperator op) {
-        int[] inputDependencyLabels = new int[] { 0 };
-        int[] outputDependencyLabels;
-        if (isSingleActivity) {
-            outputDependencyLabels = new int[] { 0 };
-        } else {
-            outputDependencyLabels = new int[] { 1 };
-        }
-        return new Pair<int[], int[]>(inputDependencyLabels, outputDependencyLabels);
-    }
-
-    @Override
-    public boolean expensiveThanMaterialization() {
-        return false;
-    }
-}
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/base/RuleCollections.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/base/RuleCollections.java
index 5cc5ede..9626774 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/base/RuleCollections.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/base/RuleCollections.java
@@ -44,7 +44,6 @@
 import edu.uci.ics.asterix.optimizer.rules.IntroduceUnnestForCollectionToSequenceRule;
 import edu.uci.ics.asterix.optimizer.rules.LoadRecordFieldsRule;
 import edu.uci.ics.asterix.optimizer.rules.NestGroupByRule;
-import edu.uci.ics.asterix.optimizer.rules.NestedSubplanToJoinRule;
 import edu.uci.ics.asterix.optimizer.rules.PushAggFuncIntoStandaloneAggregateRule;
 import edu.uci.ics.asterix.optimizer.rules.PushAggregateIntoGroupbyRule;
 import edu.uci.ics.asterix.optimizer.rules.PushFieldAccessRule;
@@ -92,9 +91,10 @@
 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.LeftOuterJoinToInnerJoinRule;
+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.PushAssignBelowUnionAllRule;
-import edu.uci.ics.hyracks.algebricks.rewriter.rules.PushAssignDownThroughProductRule;
+import edu.uci.ics.hyracks.algebricks.rewriter.rules.PushMapOperatorDownThroughProductRule;
 import edu.uci.ics.hyracks.algebricks.rewriter.rules.PushGroupByIntoSortRule;
 import edu.uci.ics.hyracks.algebricks.rewriter.rules.PushNestedOrderByUnderPreSortedGroupByRule;
 import edu.uci.ics.hyracks.algebricks.rewriter.rules.PushProjectDownRule;
@@ -102,7 +102,6 @@
 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.PushUnnestDownThroughProductRule;
 import edu.uci.ics.hyracks.algebricks.rewriter.rules.ReinferAllTypesRule;
 import edu.uci.ics.hyracks.algebricks.rewriter.rules.RemoveRedundantGroupByDecorVars;
 import edu.uci.ics.hyracks.algebricks.rewriter.rules.RemoveRedundantVariablesRule;
@@ -166,8 +165,7 @@
         condPushDownAndJoinInference.add(new DisjunctivePredicateToJoinRule());
         condPushDownAndJoinInference.add(new PushSelectIntoJoinRule());
         condPushDownAndJoinInference.add(new IntroJoinInsideSubplanRule());
-        condPushDownAndJoinInference.add(new PushAssignDownThroughProductRule());
-        condPushDownAndJoinInference.add(new PushUnnestDownThroughProductRule());
+        condPushDownAndJoinInference.add(new PushMapOperatorDownThroughProductRule());
         condPushDownAndJoinInference.add(new PushSubplanWithAggregateDownThroughProductRule());
         condPushDownAndJoinInference.add(new IntroduceGroupByForSubplanRule());
         condPushDownAndJoinInference.add(new SubplanOutOfGroupRule());
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/IntroduceMaterializationForInsertWithSelfScanRule.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/IntroduceMaterializationForInsertWithSelfScanRule.java
index c53e49a..06b203c 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/IntroduceMaterializationForInsertWithSelfScanRule.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/IntroduceMaterializationForInsertWithSelfScanRule.java
@@ -17,8 +17,6 @@
 import org.apache.commons.lang3.mutable.Mutable;
 import org.apache.commons.lang3.mutable.MutableObject;
 
-import edu.uci.ics.asterix.algebra.operators.MaterializeOperator;
-import edu.uci.ics.asterix.algebra.operators.physical.MaterializePOperator;
 import edu.uci.ics.asterix.metadata.declared.AqlDataSource;
 import edu.uci.ics.asterix.metadata.declared.AqlDataSource.AqlDataSourceType;
 import edu.uci.ics.asterix.metadata.declared.DatasetDataSource;
@@ -34,9 +32,10 @@
 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.algebricks.core.algebra.operators.logical.DataSourceScanOperator;
-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ExtensionOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.InsertDeleteOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.MaterializeOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.UnnestMapOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical.MaterializePOperator;
 import edu.uci.ics.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
 
 public class IntroduceMaterializationForInsertWithSelfScanRule implements IAlgebraicRewriteRule {
@@ -63,15 +62,12 @@
             MaterializePOperator materializePOperator = new MaterializePOperator(true);
             materializeOperator.setPhysicalOperator(materializePOperator);
 
-            ExtensionOperator extensionOperator = new ExtensionOperator(materializeOperator);
-            extensionOperator.setPhysicalOperator(materializePOperator);
-
-            extensionOperator.getInputs().add(
+            materializeOperator.getInputs().add(
                     new MutableObject<ILogicalOperator>(insertOp.getInputs().get(0).getValue()));
-            context.computeAndSetTypeEnvironmentForOperator(extensionOperator);
+            context.computeAndSetTypeEnvironmentForOperator(materializeOperator);
 
             insertOp.getInputs().clear();
-            insertOp.getInputs().add(new MutableObject<ILogicalOperator>(extensionOperator));
+            insertOp.getInputs().add(new MutableObject<ILogicalOperator>(materializeOperator));
             context.computeAndSetTypeEnvironmentForOperator(insertOp);
             return true;
         } else {
@@ -105,7 +101,8 @@
             } else if (descendantOp.getOperatorTag() == LogicalOperatorTag.DATASOURCESCAN) {
                 DataSourceScanOperator dataSourceScanOp = (DataSourceScanOperator) descendantOp;
                 AqlDataSource ds = (AqlDataSource) dataSourceScanOp.getDataSource();
-                if (ds.getDatasourceType() != AqlDataSourceType.FEED && ds.getDatasourceType() != AqlDataSourceType.ADAPTED_LOADABLE) {
+                if (ds.getDatasourceType() != AqlDataSourceType.FEED
+                        && ds.getDatasourceType() != AqlDataSourceType.ADAPTED_LOADABLE) {
                     if (((DatasetDataSource) ds).getDataset().getDatasetName().compareTo(insertDatasetName) == 0) {
                         return true;
                     }
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/NestedSubplanToJoinRule.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/NestedSubplanToJoinRule.java
deleted file mode 100644
index 6929158..0000000
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/NestedSubplanToJoinRule.java
+++ /dev/null
@@ -1,137 +0,0 @@
-/*
- * Copyright 2009-2013 by The Regents of the University of California
- * Licensed 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 from
- * 
- *     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 edu.uci.ics.asterix.optimizer.rules;
-
-import java.util.ArrayList;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Set;
-
-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.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.expressions.ConstantExpression;
-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.EmptyTupleSourceOperator;
-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.LeftOuterJoinOperator;
-import edu.uci.ics.hyracks.algebricks.core.algebra.util.OperatorPropertiesUtil;
-import edu.uci.ics.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
-
-/**
- * replace Subplan operators with nested loop joins where the join condition is true, if the Subplan
- * does not contain free variables (does not have correlations to the input stream).
- * 
- * @author yingyib
- */
-public class NestedSubplanToJoinRule 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 {
-        if (context.checkIfInDontApplySet(this, opRef.getValue()))
-            return false;
-        context.addToDontApplySet(this, opRef.getValue());
-
-        ILogicalOperator op1 = opRef.getValue();
-        if (op1.getInputs().size() == 0) {
-            return false;
-        }
-
-        boolean rewritten = false;
-        for (int index = 0; index < op1.getInputs().size(); index++) {
-            AbstractLogicalOperator child = (AbstractLogicalOperator) op1.getInputs().get(index).getValue();
-            if (child.getOperatorTag() != LogicalOperatorTag.SUBPLAN) {
-                continue;
-            }
-
-            AbstractOperatorWithNestedPlans subplan = (AbstractOperatorWithNestedPlans) child;
-            Set<LogicalVariable> freeVars = new HashSet<LogicalVariable>();
-            OperatorPropertiesUtil.getFreeVariablesInSubplans(subplan, freeVars);
-            if (!freeVars.isEmpty()) {
-                /**
-                 * the subplan is correlated with the outer plan, other rules can deal with it
-                 */
-                continue;
-            }
-
-            /** get the input operator of the subplan operator */
-            ILogicalOperator subplanInput = subplan.getInputs().get(0).getValue();
-            AbstractLogicalOperator subplanInputOp = (AbstractLogicalOperator) subplanInput;
-
-            /** If the other join branch is a trivial plan, do not do the rewriting. */
-            if (subplanInputOp.getOperatorTag() == LogicalOperatorTag.EMPTYTUPLESOURCE) {
-                continue;
-            }
-
-            /** get all nested top operators */
-            List<ILogicalPlan> nestedPlans = subplan.getNestedPlans();
-            List<Mutable<ILogicalOperator>> nestedRoots = new ArrayList<Mutable<ILogicalOperator>>();
-            for (ILogicalPlan nestedPlan : nestedPlans) {
-                nestedRoots.addAll(nestedPlan.getRoots());
-            }
-            if (nestedRoots.size() == 0) {
-                /** there is no nested top operators */
-                continue;
-            }
-
-            /** expend the input and roots into a DAG of nested loop joins */
-            Mutable<ILogicalExpression> expr = new MutableObject<ILogicalExpression>(ConstantExpression.TRUE);
-            Mutable<ILogicalOperator> nestedRootRef = nestedRoots.get(0);
-            ILogicalOperator join = new LeftOuterJoinOperator(expr, new MutableObject<ILogicalOperator>(subplanInput),
-                    nestedRootRef);
-
-            /** rewrite the nested tuple source to be empty tuple source */
-            rewriteNestedTupleSource(nestedRootRef);
-
-            for (int i = 1; i < nestedRoots.size(); i++) {
-                join = new LeftOuterJoinOperator(expr, new MutableObject<ILogicalOperator>(join), nestedRoots.get(i));
-            }
-            op1.getInputs().get(index).setValue(join);
-            context.computeAndSetTypeEnvironmentForOperator(join);
-            rewritten = true;
-        }
-        return rewritten;
-    }
-
-    /**
-     * rewrite NestedTupleSource operators to EmptyTupleSource operators
-     * 
-     * @param nestedRootRef
-     */
-    private void rewriteNestedTupleSource(Mutable<ILogicalOperator> nestedRootRef) {
-        AbstractLogicalOperator nestedRoot = (AbstractLogicalOperator) nestedRootRef.getValue();
-        if (nestedRoot.getOperatorTag() == LogicalOperatorTag.NESTEDTUPLESOURCE) {
-            nestedRootRef.setValue(new EmptyTupleSourceOperator());
-        }
-        List<Mutable<ILogicalOperator>> inputs = nestedRoot.getInputs();
-        for (Mutable<ILogicalOperator> input : inputs) {
-            rewriteNestedTupleSource(input);
-        }
-    }
-}
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/SweepIllegalNonfunctionalFunctions.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/SweepIllegalNonfunctionalFunctions.java
index cfd25c0..3ebe6b6 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/SweepIllegalNonfunctionalFunctions.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/SweepIllegalNonfunctionalFunctions.java
@@ -27,6 +27,7 @@
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.InsertDeleteOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.LeftOuterJoinOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.LimitOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.MaterializeOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.NestedTupleSourceOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.OrderOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.OrderOperator.IOrder;
@@ -189,6 +190,11 @@
         }
 
         @Override
+        public Void visitMaterializeOperator(MaterializeOperator op, Void arg) throws AlgebricksException {
+            return null;
+        }
+
+        @Override
         public Void visitScriptOperator(ScriptOperator op, Void arg) throws AlgebricksException {
             return null;
         }
diff --git a/pom.xml b/pom.xml
index 95f7fc9..e63c417 100644
--- a/pom.xml
+++ b/pom.xml
@@ -13,157 +13,160 @@
  ! See the License for the specific language governing permissions and
  ! limitations under the License.
  !-->
-<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
-  <modelVersion>4.0.0</modelVersion>
-  <groupId>edu.uci.ics.asterix</groupId>
-  <artifactId>asterix</artifactId>
-  <version>0.8.7-SNAPSHOT</version>
-  <packaging>pom</packaging>
-  
-  <properties>
-    <project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
-    <jvm.extraargs />
-    <runSlowAQLTests>false</runSlowAQLTests>
+<project
+    xmlns="http://maven.apache.org/POM/4.0.0"
+    xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+    xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
+    <modelVersion>4.0.0</modelVersion>
+    <groupId>edu.uci.ics.asterix</groupId>
+    <artifactId>asterix</artifactId>
+    <version>0.8.7-SNAPSHOT</version>
+    <packaging>pom</packaging>
+
+    <properties>
+        <project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
+        <jvm.extraargs />
+        <runSlowAQLTests>false</runSlowAQLTests>
 
     <!-- Definition of tests in various categories which may be excluded -->
-    <optimizer.tests>**/optimizer/**/*Test.java</optimizer.tests>
-    <metadata.tests>**/metadata/*Test.java</metadata.tests>
-    <execution.tests>**/ExecutionTest.java</execution.tests>
-    <invalid.tests>**/DmlTest.java</invalid.tests>
-    <global.test.includes>**/*TestSuite.java,**/*Test.java,${execution.tests}</global.test.includes>
-    <global.test.excludes>${optimizer.tests},${metadata.tests},${invalid.tests}</global.test.excludes>
+        <optimizer.tests>**/optimizer/**/*Test.java</optimizer.tests>
+        <metadata.tests>**/metadata/*Test.java</metadata.tests>
+        <execution.tests>**/ExecutionTest.java</execution.tests>
+        <invalid.tests>**/DmlTest.java</invalid.tests>
+        <global.test.includes>**/*TestSuite.java,**/*Test.java,${execution.tests}</global.test.includes>
+        <global.test.excludes>${optimizer.tests},${metadata.tests},${invalid.tests}</global.test.excludes>
     <!-- Versions under dependencymanagement or used in many projects via properties -->
-    <algebricks.version>0.2.15-SNAPSHOT</algebricks.version>
-    <hyracks.version>0.2.15-SNAPSHOT</hyracks.version>
-    <hadoop.version>2.2.0</hadoop.version>
-    <junit.version>4.8.1</junit.version>
-    <commons.io.version>2.4</commons.io.version>
-    <servlet.api.version>2.5</servlet.api.version>
-    <json.version>20090211</json.version>
-  </properties>
+        <algebricks.version>0.2.15-SNAPSHOT</algebricks.version>
+        <hyracks.version>0.2.15-SNAPSHOT</hyracks.version>
+        <hadoop.version>2.2.0</hadoop.version>
+        <junit.version>4.8.1</junit.version>
+        <commons.io.version>2.4</commons.io.version>
+        <servlet.api.version>2.5</servlet.api.version>
+        <json.version>20090211</json.version>
+    </properties>
 
-	<build>
-	  <plugins>
-	    <plugin>
-	      <groupId>org.apache.maven.plugins</groupId>
-	      <artifactId>maven-release-plugin</artifactId>
-	      <version>2.1</version>
-	      <configuration>
-                <goals>package source:jar javadoc:jar deploy:deploy</goals>
-	      </configuration>
-	    </plugin>
-	    <plugin>
-	      <groupId>org.apache.maven.plugins</groupId>
-	      <artifactId>maven-surefire-plugin</artifactId>
-	      <version>2.16</version>
-	      <configuration>
-		<failIfNoTests>false</failIfNoTests>
-		<systemPropertyVariables>
-                  <skipFredSlowTests>true</skipFredSlowTests>
-		</systemPropertyVariables>
-		<forkCount>1</forkCount>
-                <reuseForks>false</reuseForks>
-		<argLine>-enableassertions -Xmx${test.heap.size}m
-		-Dfile.encoding=UTF-8
-		-Djava.util.logging.config.file=${user.home}/logging.properties
-		-DrunSlowAQLTests=${runSlowAQLTests}
-		-Xdebug
-		-Xrunjdwp:transport=dt_socket,server=y,address=8000,suspend=n</argLine>
-		<includes>
-		  <include>${global.test.includes},${test.includes}</include>
-		</includes>
-                <excludes>
-                  <exclude>${global.test.excludes},${test.excludes}</exclude>
-                </excludes>
-	      </configuration>
-	    </plugin>
-	  </plugins>
-	</build>
+    <build>
+        <plugins>
+            <plugin>
+                <groupId>org.apache.maven.plugins</groupId>
+                <artifactId>maven-release-plugin</artifactId>
+                <version>2.1</version>
+                <configuration>
+                    <goals>package source:jar javadoc:jar deploy:deploy</goals>
+                </configuration>
+            </plugin>
+            <plugin>
+                <groupId>org.apache.maven.plugins</groupId>
+                <artifactId>maven-surefire-plugin</artifactId>
+                <version>2.16</version>
+                <configuration>
+                    <failIfNoTests>false</failIfNoTests>
+                    <systemPropertyVariables>
+                        <skipFredSlowTests>true</skipFredSlowTests>
+                    </systemPropertyVariables>
+                    <forkCount>1</forkCount>
+                    <reuseForks>false</reuseForks>
+                    <argLine>-enableassertions -Xmx${test.heap.size}m
+                        -Dfile.encoding=UTF-8
+                        -Djava.util.logging.config.file=${user.home}/logging.properties
+                        -DrunSlowAQLTests=${runSlowAQLTests}
+                        -Xdebug
+                        -Xrunjdwp:transport=dt_socket,server=y,address=8000,suspend=n</argLine>
+                    <includes>
+                        <include>${global.test.includes},${test.includes}</include>
+                    </includes>
+                    <excludes>
+                        <exclude>${global.test.excludes},${test.excludes}</exclude>
+                    </excludes>
+                </configuration>
+            </plugin>
+        </plugins>
+    </build>
 
-	<scm>
-		<connection>scm:git:https://code.google.com/p/asterixdb/</connection>
-		<developerConnection>scm:git:ssh://fulliautomatix.ics.uci.edu:29418/asterixdb</developerConnection>
-	</scm>
+    <scm>
+        <connection>scm:git:https://code.google.com/p/asterixdb/</connection>
+        <developerConnection>scm:git:ssh://fulliautomatix.ics.uci.edu:29418/asterixdb</developerConnection>
+    </scm>
 
-	<distributionManagement>
-		<repository>
-			<id>asterix-releases</id>
-			<url>http://obelix.ics.uci.edu/nexus/content/repositories/asterix-releases/</url>
-		</repository>
-		<snapshotRepository>
-			<id>asterix-snapshots</id>
-			<url>http://obelix.ics.uci.edu/nexus/content/repositories/asterix-snapshots/</url>
-		</snapshotRepository>
-	</distributionManagement>
+    <distributionManagement>
+        <repository>
+            <id>asterix-releases</id>
+            <url>http://obelix.ics.uci.edu/nexus/content/repositories/asterix-releases/</url>
+        </repository>
+        <snapshotRepository>
+            <id>asterix-snapshots</id>
+            <url>http://obelix.ics.uci.edu/nexus/content/repositories/asterix-snapshots/</url>
+        </snapshotRepository>
+    </distributionManagement>
 
     <profiles>
-      <profile>
-        <id>slow-aql-tests</id>
-        <properties>
-          <runSlowAQLTests>true</runSlowAQLTests>
-        </properties>
-      </profile>
-      <profile>
-        <id>optimizer-tests</id>
-        <properties>
-          <optimizer.tests />
-        </properties>
-      </profile>
-      <profile>
-        <id>metadata-tests</id>
-        <properties>
-          <metadata.tests />
-        </properties>
-      </profile>
-      <profile>
-        <id>execution-tests</id>
-        <properties>
-          <execution.tests />
-        </properties>
-      </profile>
-      <profile>
-        <id>invalid-tests</id>
-        <properties>
-          <invalid.tests />
-        </properties>
-      </profile>
         <profile>
-          <id>32bitvm</id>
-          <activation>
-            <property>
-              <name>sun.arch.data.model</name>
-              <value>32</value>
-            </property>
-          </activation>
-          <properties>
-            <test.heap.size>2048</test.heap.size>
-          </properties>
+            <id>slow-aql-tests</id>
+            <properties>
+                <runSlowAQLTests>true</runSlowAQLTests>
+            </properties>
+        </profile>
+        <profile>
+            <id>optimizer-tests</id>
+            <properties>
+                <optimizer.tests />
+            </properties>
+        </profile>
+        <profile>
+            <id>metadata-tests</id>
+            <properties>
+                <metadata.tests />
+            </properties>
+        </profile>
+        <profile>
+            <id>execution-tests</id>
+            <properties>
+                <execution.tests />
+            </properties>
+        </profile>
+        <profile>
+            <id>invalid-tests</id>
+            <properties>
+                <invalid.tests />
+            </properties>
+        </profile>
+        <profile>
+            <id>32bitvm</id>
+            <activation>
+                <property>
+                    <name>sun.arch.data.model</name>
+                    <value>32</value>
+                </property>
+            </activation>
+            <properties>
+                <test.heap.size>2048</test.heap.size>
+            </properties>
         </profile>
 
         <profile>
-          <id>64bitvm</id>
-          <activation>
-            <property>
-              <name>sun.arch.data.model</name>
-              <value>64</value>
-            </property>
-          </activation>
-          <properties>
-            <test.heap.size>3072</test.heap.size>
-          </properties>
+            <id>64bitvm</id>
+            <activation>
+                <property>
+                    <name>sun.arch.data.model</name>
+                    <value>64</value>
+                </property>
+            </activation>
+            <properties>
+                <test.heap.size>3072</test.heap.size>
+            </properties>
         </profile>
 
     </profiles>
 
-	<modules>
-		<module>asterix-common</module>
-		<module>asterix-algebra</module>
-		<module>asterix-app</module>
-		<module>asterix-tools</module>
-		<module>asterix-transactions</module>
-		<module>asterix-runtime</module>
-		<module>asterix-om</module>
-		<module>asterix-aql</module>
+    <modules>
+        <module>asterix-common</module>
+        <module>asterix-algebra</module>
+        <module>asterix-app</module>
+        <module>asterix-tools</module>
+        <module>asterix-transactions</module>
+        <module>asterix-runtime</module>
+        <module>asterix-om</module>
+        <module>asterix-aql</module>
         <module>asterix-external-data</module>
         <module>asterix-examples</module>
         <module>asterix-metadata</module>
@@ -174,197 +177,197 @@
         <module>asterix-events</module>
         <module>asterix-doc</module>
         <module>asterix-fuzzyjoin</module>
-        </modules>
+    </modules>
 
-	<repositories>
-		<repository>
-			<releases>
-				<enabled>true</enabled>
-				<updatePolicy>always</updatePolicy>
-				<checksumPolicy>warn</checksumPolicy>
-			</releases>
-			<snapshots>
-				<enabled>true</enabled>
-				<updatePolicy>always</updatePolicy>
-				<checksumPolicy>fail</checksumPolicy>
-			</snapshots>
-			<id>asterix-public</id>
-			<url>http://obelix.ics.uci.edu/nexus/content/groups/asterix-public/</url>
-		</repository>
-		<repository>
-			<releases>
-				<enabled>true</enabled>
-				<updatePolicy>always</updatePolicy>
-				<checksumPolicy>warn</checksumPolicy>
-			</releases>
-			<snapshots>
-				<enabled>true</enabled>
-				<updatePolicy>always</updatePolicy>
-				<checksumPolicy>fail</checksumPolicy>
-			</snapshots>
-			<id>third-party</id>
-			<url>http://obelix.ics.uci.edu/nexus/content/repositories/third-party/</url>
-		</repository>
-		<repository>
-			<releases>
-				<enabled>true</enabled>
-				<updatePolicy>always</updatePolicy>
-				<checksumPolicy>warn</checksumPolicy>
-			</releases>
-			<id>algebricks-releases</id>
-			<url>http://obelix.ics.uci.edu/nexus/content/repositories/algebricks-releases/</url>
-		</repository>
-		<repository>
-			<snapshots>
-				<enabled>true</enabled>
-				<updatePolicy>always</updatePolicy>
-				<checksumPolicy>fail</checksumPolicy>
-			</snapshots>
-			<id>algebricks-snapshots</id>
-			<url>http://obelix.ics.uci.edu/nexus/content/repositories/algebricks-snapshots/</url>
-		</repository>
-	</repositories>
-        <dependencyManagement>
-    	<dependencies>
-          <dependency>
-            <groupId>junit</groupId>
-            <artifactId>junit</artifactId>
-            <version>${junit.version}</version>
-          </dependency>
-          <dependency>
-            <groupId>org.apache.maven</groupId>
-            <artifactId>maven-plugin-api</artifactId>
-            <version>2.2.1</version>
-          </dependency>
-          <dependency>
-            <groupId>org.apache.hadoop</groupId>
-            <artifactId>hadoop-yarn-common</artifactId>
-            <version>${hadoop.version}</version>
-          </dependency>
-          <dependency>
-            <groupId>org.apache.hadoop</groupId>
-            <artifactId>hadoop-yarn-client</artifactId>
-            <version>${hadoop.version}</version>
-          </dependency>
-          <dependency>
-            <groupId>org.apache.hadoop</groupId>
-            <artifactId>hadoop-client</artifactId>
-            <version>${hadoop.version}</version>
-          </dependency>
-          <dependency>
-            <groupId>org.apache.hadoop</groupId>
-            <artifactId>hadoop-hdfs</artifactId>
-            <version>${hadoop.version}</version>
-          </dependency>
-		  <dependency>
-			<groupId>org.apache.hadoop</groupId>
-			<artifactId>hadoop-common</artifactId>
-			<version>${hadoop.version}</version>
-		  </dependency>
-    		<dependency>
-    			<groupId>edu.uci.ics.hyracks</groupId>
-    			<artifactId>algebricks-compiler</artifactId>
-    			<version>${algebricks.version}</version>
-    		</dependency>
-    		<dependency>
-    			<groupId>edu.uci.ics.hyracks</groupId>
-    			<artifactId>hyracks-api</artifactId>
-    			<version>${hyracks.version}</version>
-    		</dependency>
-    		<dependency>
-    			<groupId>edu.uci.ics.hyracks</groupId>
-    			<artifactId>hyracks-dataflow-std</artifactId>
-    			<version>${hyracks.version}</version>
-    		</dependency>
-    		<dependency>
-    			<groupId>edu.uci.ics.hyracks</groupId>
-    			<artifactId>hyracks-control-cc</artifactId>
-    			<version>${hyracks.version}</version>
-    		</dependency>
-    		<dependency>
-    			<groupId>edu.uci.ics.hyracks</groupId>
-    			<artifactId>hyracks-control-nc</artifactId>
-    			<version>${hyracks.version}</version>
-    		</dependency>
-    		<dependency>
-    			<groupId>edu.uci.ics.hyracks</groupId>
-    			<artifactId>hyracks-server</artifactId>
-    			<version>${hyracks.version}</version>
-    		</dependency>
-    		<dependency>
-    			<groupId>edu.uci.ics.hyracks</groupId>
-    			<artifactId>hyracks-cli</artifactId>
-    			<version>${hyracks.version}</version>
-    		</dependency>
-    		<dependency>
-    			<groupId>edu.uci.ics.hyracks</groupId>
-    			<artifactId>hyracks-dataflow-hadoop</artifactId>
-    			<version>${hyracks.version}</version>
-    		</dependency>
-    		<dependency>
-    			<groupId>edu.uci.ics.hyracks</groupId>
-    			<artifactId>hyracks-storage-am-btree</artifactId>
-    			<version>${hyracks.version}</version>
-    		</dependency>
-    		<dependency>
-    			<groupId>edu.uci.ics.hyracks</groupId>
-    			<artifactId>hyracks-storage-am-rtree</artifactId>
-    			<version>${hyracks.version}</version>
-    		</dependency>
-    		<dependency>
-    			<groupId>edu.uci.ics.hyracks</groupId>
-    			<artifactId>
+    <repositories>
+        <repository>
+            <releases>
+                <enabled>true</enabled>
+                <updatePolicy>always</updatePolicy>
+                <checksumPolicy>warn</checksumPolicy>
+            </releases>
+            <snapshots>
+                <enabled>true</enabled>
+                <updatePolicy>always</updatePolicy>
+                <checksumPolicy>fail</checksumPolicy>
+            </snapshots>
+            <id>asterix-public</id>
+            <url>http://obelix.ics.uci.edu/nexus/content/groups/asterix-public/</url>
+        </repository>
+        <repository>
+            <releases>
+                <enabled>true</enabled>
+                <updatePolicy>always</updatePolicy>
+                <checksumPolicy>warn</checksumPolicy>
+            </releases>
+            <snapshots>
+                <enabled>true</enabled>
+                <updatePolicy>always</updatePolicy>
+                <checksumPolicy>fail</checksumPolicy>
+            </snapshots>
+            <id>third-party</id>
+            <url>http://obelix.ics.uci.edu/nexus/content/repositories/third-party/</url>
+        </repository>
+        <repository>
+            <releases>
+                <enabled>true</enabled>
+                <updatePolicy>always</updatePolicy>
+                <checksumPolicy>warn</checksumPolicy>
+            </releases>
+            <id>algebricks-releases</id>
+            <url>http://obelix.ics.uci.edu/nexus/content/repositories/algebricks-releases/</url>
+        </repository>
+        <repository>
+            <snapshots>
+                <enabled>true</enabled>
+                <updatePolicy>always</updatePolicy>
+                <checksumPolicy>fail</checksumPolicy>
+            </snapshots>
+            <id>algebricks-snapshots</id>
+            <url>http://obelix.ics.uci.edu/nexus/content/repositories/algebricks-snapshots/</url>
+        </repository>
+    </repositories>
+    <dependencyManagement>
+        <dependencies>
+            <dependency>
+                <groupId>junit</groupId>
+                <artifactId>junit</artifactId>
+                <version>${junit.version}</version>
+            </dependency>
+            <dependency>
+                <groupId>org.apache.maven</groupId>
+                <artifactId>maven-plugin-api</artifactId>
+                <version>2.2.1</version>
+            </dependency>
+            <dependency>
+                <groupId>org.apache.hadoop</groupId>
+                <artifactId>hadoop-yarn-common</artifactId>
+                <version>${hadoop.version}</version>
+            </dependency>
+            <dependency>
+                <groupId>org.apache.hadoop</groupId>
+                <artifactId>hadoop-yarn-client</artifactId>
+                <version>${hadoop.version}</version>
+            </dependency>
+            <dependency>
+                <groupId>org.apache.hadoop</groupId>
+                <artifactId>hadoop-client</artifactId>
+                <version>${hadoop.version}</version>
+            </dependency>
+            <dependency>
+                <groupId>org.apache.hadoop</groupId>
+                <artifactId>hadoop-hdfs</artifactId>
+                <version>${hadoop.version}</version>
+            </dependency>
+            <dependency>
+                <groupId>org.apache.hadoop</groupId>
+                <artifactId>hadoop-common</artifactId>
+                <version>${hadoop.version}</version>
+            </dependency>
+            <dependency>
+                <groupId>edu.uci.ics.hyracks</groupId>
+                <artifactId>algebricks-compiler</artifactId>
+                <version>${algebricks.version}</version>
+            </dependency>
+            <dependency>
+                <groupId>edu.uci.ics.hyracks</groupId>
+                <artifactId>hyracks-api</artifactId>
+                <version>${hyracks.version}</version>
+            </dependency>
+            <dependency>
+                <groupId>edu.uci.ics.hyracks</groupId>
+                <artifactId>hyracks-dataflow-std</artifactId>
+                <version>${hyracks.version}</version>
+            </dependency>
+            <dependency>
+                <groupId>edu.uci.ics.hyracks</groupId>
+                <artifactId>hyracks-control-cc</artifactId>
+                <version>${hyracks.version}</version>
+            </dependency>
+            <dependency>
+                <groupId>edu.uci.ics.hyracks</groupId>
+                <artifactId>hyracks-control-nc</artifactId>
+                <version>${hyracks.version}</version>
+            </dependency>
+            <dependency>
+                <groupId>edu.uci.ics.hyracks</groupId>
+                <artifactId>hyracks-server</artifactId>
+                <version>${hyracks.version}</version>
+            </dependency>
+            <dependency>
+                <groupId>edu.uci.ics.hyracks</groupId>
+                <artifactId>hyracks-cli</artifactId>
+                <version>${hyracks.version}</version>
+            </dependency>
+            <dependency>
+                <groupId>edu.uci.ics.hyracks</groupId>
+                <artifactId>hyracks-dataflow-hadoop</artifactId>
+                <version>${hyracks.version}</version>
+            </dependency>
+            <dependency>
+                <groupId>edu.uci.ics.hyracks</groupId>
+                <artifactId>hyracks-storage-am-btree</artifactId>
+                <version>${hyracks.version}</version>
+            </dependency>
+            <dependency>
+                <groupId>edu.uci.ics.hyracks</groupId>
+                <artifactId>hyracks-storage-am-rtree</artifactId>
+                <version>${hyracks.version}</version>
+            </dependency>
+            <dependency>
+                <groupId>edu.uci.ics.hyracks</groupId>
+                <artifactId>
     				hyracks-storage-am-invertedindex
     			</artifactId>
-    			<version>${hyracks.version}</version>
-    		</dependency>
-    		<dependency>
-    			<groupId>edu.uci.ics.hyracks</groupId>
-    			<artifactId>hyracks-storage-am-common</artifactId>
-    			<version>${hyracks.version}</version>
-    		</dependency>
-		    <dependency>
+                <version>${hyracks.version}</version>
+            </dependency>
+            <dependency>
+                <groupId>edu.uci.ics.hyracks</groupId>
+                <artifactId>hyracks-storage-am-common</artifactId>
+                <version>${hyracks.version}</version>
+            </dependency>
+            <dependency>
                 <groupId>edu.uci.ics.hyracks</groupId>
                 <artifactId>hyracks-client</artifactId>
                 <version>${hyracks.version}</version>
             </dependency>
-		<dependency>
-			<groupId>edu.uci.ics.hyracks</groupId>
-			<artifactId>hyracks-storage-am-lsm-common</artifactId>
-			<version>${hyracks.version}</version>
-		</dependency>
-		<dependency>
-			<groupId>edu.uci.ics.hyracks</groupId>
-			<artifactId>hyracks-storage-am-lsm-btree</artifactId>
-			<version>${hyracks.version}</version>
-		</dependency>
-		<dependency>
-			<groupId>edu.uci.ics.hyracks</groupId>
-			<artifactId>hyracks-storage-am-lsm-rtree</artifactId>
-			<version>${hyracks.version}</version>
-		</dependency>
-		<dependency>
-			<groupId>edu.uci.ics.hyracks</groupId>
-			<artifactId>hyracks-storage-am-lsm-invertedindex</artifactId>
-			<version>${hyracks.version}</version>
-		</dependency>
-		<dependency>
-			<groupId>org.json</groupId>
-			<artifactId>json</artifactId>
-			<version>${json.version}</version>
-			<type>jar</type>
-		</dependency>
-		<dependency>
-			<groupId>javax.servlet</groupId>
-			<artifactId>servlet-api</artifactId>
-			<version>${servlet.api.version}</version>
-			<type>jar</type>
-		</dependency>
-        <dependency>
-            <groupId>commons-io</groupId>
-            <artifactId>commons-io</artifactId>
-            <version>${commons.io.version}</version>
-        </dependency>
-    	</dependencies>
+            <dependency>
+                <groupId>edu.uci.ics.hyracks</groupId>
+                <artifactId>hyracks-storage-am-lsm-common</artifactId>
+                <version>${hyracks.version}</version>
+            </dependency>
+            <dependency>
+                <groupId>edu.uci.ics.hyracks</groupId>
+                <artifactId>hyracks-storage-am-lsm-btree</artifactId>
+                <version>${hyracks.version}</version>
+            </dependency>
+            <dependency>
+                <groupId>edu.uci.ics.hyracks</groupId>
+                <artifactId>hyracks-storage-am-lsm-rtree</artifactId>
+                <version>${hyracks.version}</version>
+            </dependency>
+            <dependency>
+                <groupId>edu.uci.ics.hyracks</groupId>
+                <artifactId>hyracks-storage-am-lsm-invertedindex</artifactId>
+                <version>${hyracks.version}</version>
+            </dependency>
+            <dependency>
+                <groupId>org.json</groupId>
+                <artifactId>json</artifactId>
+                <version>${json.version}</version>
+                <type>jar</type>
+            </dependency>
+            <dependency>
+                <groupId>javax.servlet</groupId>
+                <artifactId>servlet-api</artifactId>
+                <version>${servlet.api.version}</version>
+                <type>jar</type>
+            </dependency>
+            <dependency>
+                <groupId>commons-io</groupId>
+                <artifactId>commons-io</artifactId>
+                <version>${commons.io.version}</version>
+            </dependency>
+        </dependencies>
     </dependencyManagement>
 </project>