Adding external indexes

In Hyracks side, this change include the following:
1. The addition of three indexes:
 a) external b-tree index
 b) external r-tree index
 c) external b-tree with buddy b-tree index
2. creating an additional logical operator in algebricks for performing lookup operations over external data and modify the different visitors to work with this operator

3. Added copyright header to all new files

Change-Id: Iecfbd86f06aff3caaf3a9652b63420666745ebb9
Reviewed-on: http://fulliautomatix.ics.uci.edu:8443/69
Tested-by: Jenkins <jenkins@fulliautomatix.ics.uci.edu>
Reviewed-by: Zachary Heilbron <zheilbron@gmail.com>
Reviewed-by: Sattam Alsubaiee <salsubaiee@gmail.com>
Reviewed-by: Ian Maxon <imaxon@uci.edu>
diff --git a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/base/LogicalOperatorTag.java b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/base/LogicalOperatorTag.java
index bee4906..e15ffe0 100644
--- a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/base/LogicalOperatorTag.java
+++ b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/base/LogicalOperatorTag.java
@@ -45,5 +45,6 @@
     INSERT_DELETE,
     INDEX_INSERT_DELETE,
     UPDATE,
-    EXTENSION_OPERATOR
+    EXTENSION_OPERATOR,
+    EXTERNAL_LOOKUP
 }
\ No newline at end of file
diff --git a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/base/PhysicalOperatorTag.java b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/base/PhysicalOperatorTag.java
index f6de971..c41107b 100644
--- a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/base/PhysicalOperatorTag.java
+++ b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/base/PhysicalOperatorTag.java
@@ -61,5 +61,6 @@
     SINGLE_PARTITION_INVERTED_INDEX_SEARCH,
     LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH,
     PARTITIONINGSPLIT,
-    EXTENSION_OPERATOR
+    EXTENSION_OPERATOR,
+    EXTERNAL_LOOKUP
 }
diff --git a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/ExternalDataLookupOperator.java b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/ExternalDataLookupOperator.java
new file mode 100644
index 0000000..e18c461
--- /dev/null
+++ b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/ExternalDataLookupOperator.java
@@ -0,0 +1,114 @@
+/*
+ * 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.hyracks.algebricks.core.algebra.operators.logical;
+
+import java.util.ArrayList;
+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.LogicalOperatorTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.visitors.VariableUtilities;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.VariablePropagationPolicy;
+import edu.uci.ics.hyracks.algebricks.core.algebra.typing.ITypingContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.typing.NonPropagatingTypeEnvironment;
+import edu.uci.ics.hyracks.algebricks.core.algebra.visitors.ILogicalExpressionReferenceTransform;
+import edu.uci.ics.hyracks.algebricks.core.algebra.visitors.ILogicalOperatorVisitor;
+
+public class ExternalDataLookupOperator extends AbstractScanOperator {
+
+    private final List<Object> variableTypes;
+    protected final Mutable<ILogicalExpression> expression;
+    private final boolean propagateInput;
+
+    public ExternalDataLookupOperator(List<LogicalVariable> variables, Mutable<ILogicalExpression> expression,
+            List<Object> variableTypes, boolean propagateInput) {
+        super(variables);
+        this.expression = expression;
+        this.variableTypes = variableTypes;
+        this.propagateInput = propagateInput;
+    }
+
+    @Override
+    public LogicalOperatorTag getOperatorTag() {
+        return LogicalOperatorTag.EXTERNAL_LOOKUP;
+    }
+
+    @Override
+    public IVariableTypeEnvironment computeOutputTypeEnvironment(ITypingContext ctx) throws AlgebricksException {
+        IVariableTypeEnvironment env = null;
+        if (propagateInput) {
+            env = createPropagatingAllInputsTypeEnvironment(ctx);
+        } else {
+            env = new NonPropagatingTypeEnvironment(ctx.getExpressionTypeComputer(), ctx.getMetadataProvider());
+        }
+        env.setVarType(variables.get(0), variableTypes.get(0));
+        return env;
+    }
+
+    public List<Object> getVariableTypes() {
+        return variableTypes;
+    }
+
+    @Override
+    public <R, T> R accept(ILogicalOperatorVisitor<R, T> visitor, T arg) throws AlgebricksException {
+        return visitor.visitExternalDataLookupOperator(this, arg);
+    }
+
+    @Override
+    public boolean isMap() {
+        return false;
+    }
+
+    public boolean isPropagateInput() {
+        return propagateInput;
+    }
+
+    @Override
+    public VariablePropagationPolicy getVariablePropagationPolicy() {
+        return new VariablePropagationPolicy() {
+            @Override
+            public void propagateVariables(IOperatorSchema target, IOperatorSchema... sources)
+                    throws AlgebricksException {
+                if (propagateInput) {
+                    ArrayList<LogicalVariable> usedVariables = new ArrayList<LogicalVariable>();
+                    VariableUtilities.getUsedVariables(ExternalDataLookupOperator.this, usedVariables);
+                    int numOfSources = sources.length;
+                    for (int i = 0; i < numOfSources; i++) {
+                        for (LogicalVariable v : sources[i]) {
+                            if (!usedVariables.contains(v)) {
+                                target.addVariable(v);
+                            }
+                        }
+                    }
+                }
+                target.addVariable(variables.get(0));
+            }
+        };
+    }
+
+    public Mutable<ILogicalExpression> getExpressionRef() {
+        return expression;
+    }
+
+    @Override
+    public boolean acceptExpressionTransform(ILogicalExpressionReferenceTransform visitor) throws AlgebricksException {
+        return visitor.transform(expression);
+    }
+}
\ No newline at end of file
diff --git a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/visitors/FDsAndEquivClassesVisitor.java b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/visitors/FDsAndEquivClassesVisitor.java
index 3a2617f..ae0346e 100644
--- a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/visitors/FDsAndEquivClassesVisitor.java
+++ b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/visitors/FDsAndEquivClassesVisitor.java
@@ -50,6 +50,7 @@
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.EmptyTupleSourceOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ExchangeOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ExtensionOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ExternalDataLookupOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.GroupByOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.IndexInsertDeleteOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.InnerJoinOperator;
@@ -647,4 +648,11 @@
         return null;
     }
 
+    @Override
+    public Void visitExternalDataLookupOperator(ExternalDataLookupOperator op, IOptimizationContext ctx)
+            throws AlgebricksException {
+        propagateFDsAndEquivClasses(op, ctx);
+        return null;
+    }
+
 }
\ No newline at end of file
diff --git a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/visitors/IsomorphismOperatorVisitor.java b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/visitors/IsomorphismOperatorVisitor.java
index 9157878..9e94a5b 100644
--- a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/visitors/IsomorphismOperatorVisitor.java
+++ b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/visitors/IsomorphismOperatorVisitor.java
@@ -41,6 +41,7 @@
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.EmptyTupleSourceOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ExchangeOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ExtensionOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ExternalDataLookupOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.GroupByOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.IndexInsertDeleteOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.InnerJoinOperator;
@@ -666,7 +667,8 @@
 
         @Override
         public ILogicalOperator visitSelectOperator(SelectOperator op, Void arg) throws AlgebricksException {
-            return new SelectOperator(deepCopyExpressionRef(op.getCondition()), op.getRetainNull(), op.getNullPlaceholderVariable());
+            return new SelectOperator(deepCopyExpressionRef(op.getCondition()), op.getRetainNull(),
+                    op.getNullPlaceholderVariable());
         }
 
         @Override
@@ -824,6 +826,21 @@
         public ILogicalOperator visitExtensionOperator(ExtensionOperator op, Void arg) throws AlgebricksException {
             return new ExtensionOperator(op.getNewInstanceOfDelegateOperator());
         }
+
+        @Override
+        public ILogicalOperator visitExternalDataLookupOperator(ExternalDataLookupOperator op, Void arg)
+                throws AlgebricksException {
+            ArrayList<LogicalVariable> newInputList = new ArrayList<LogicalVariable>();
+            newInputList.addAll(op.getVariables());
+            return new ExternalDataLookupOperator(newInputList, deepCopyExpressionRef(op.getExpressionRef()),
+                    new ArrayList<Object>(op.getVariableTypes()), op.isPropagateInput());
+        }
+    }
+
+    @Override
+    public Boolean visitExternalDataLookupOperator(ExternalDataLookupOperator op, ILogicalOperator arg)
+            throws AlgebricksException {
+        return Boolean.FALSE;
     }
 
 }
diff --git a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/visitors/IsomorphismVariableMappingVisitor.java b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/visitors/IsomorphismVariableMappingVisitor.java
index eecace7..ef91b38 100644
--- a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/visitors/IsomorphismVariableMappingVisitor.java
+++ b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/visitors/IsomorphismVariableMappingVisitor.java
@@ -40,6 +40,7 @@
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.EmptyTupleSourceOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ExchangeOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ExtensionOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ExternalDataLookupOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.GroupByOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.IndexInsertDeleteOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.InnerJoinOperator;
@@ -422,4 +423,10 @@
         return null;
     }
 
+    @Override
+    public Void visitExternalDataLookupOperator(ExternalDataLookupOperator op, ILogicalOperator arg)
+            throws AlgebricksException {
+        return null;
+    }
+
 }
diff --git a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/visitors/LogicalPropertiesVisitor.java b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/visitors/LogicalPropertiesVisitor.java
index 277b5a3..225edd6 100644
--- a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/visitors/LogicalPropertiesVisitor.java
+++ b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/visitors/LogicalPropertiesVisitor.java
@@ -32,6 +32,7 @@
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.EmptyTupleSourceOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ExchangeOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ExtensionOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ExternalDataLookupOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.GroupByOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.IndexInsertDeleteOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.InnerJoinOperator;
@@ -325,4 +326,10 @@
         return null;
     }
 
+    @Override
+    public Void visitExternalDataLookupOperator(ExternalDataLookupOperator op, IOptimizationContext arg)
+            throws AlgebricksException {
+        return null;
+    }
+
 }
diff --git a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/visitors/ProducedVariableVisitor.java b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/visitors/ProducedVariableVisitor.java
index 8673a77..3b8ed56 100644
--- a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/visitors/ProducedVariableVisitor.java
+++ b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/visitors/ProducedVariableVisitor.java
@@ -12,244 +12,251 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.visitors;

-

-import java.util.ArrayList;

-import java.util.Collection;

-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.common.utils.Pair;

-import edu.uci.ics.hyracks.algebricks.common.utils.Triple;

-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.LogicalVariable;

-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AggregateOperator;

-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AssignOperator;

-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.DataSourceScanOperator;

-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.DistinctOperator;

-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.DistributeResultOperator;

-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.EmptyTupleSourceOperator;

-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ExchangeOperator;

-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ExtensionOperator;

-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.GroupByOperator;

-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.IndexInsertDeleteOperator;

-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.InnerJoinOperator;

-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.NestedTupleSourceOperator;

-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.OrderOperator;

-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.PartitioningSplitOperator;

-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ProjectOperator;

-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ReplicateOperator;

-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.RunningAggregateOperator;

-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ScriptOperator;

-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.SelectOperator;

-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.SinkOperator;

-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.SubplanOperator;

-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.UnionAllOperator;

-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.UnnestMapOperator;

-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.UnnestOperator;

-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.WriteOperator;

-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.WriteResultOperator;

-import edu.uci.ics.hyracks.algebricks.core.algebra.visitors.ILogicalOperatorVisitor;

-

-public class ProducedVariableVisitor implements ILogicalOperatorVisitor<Void, Void> {

-

-    private Collection<LogicalVariable> producedVariables;

-

-    public ProducedVariableVisitor(Collection<LogicalVariable> producedVariables) throws AlgebricksException {

-        this.producedVariables = producedVariables;

-    }

-

-    @Override

-    public Void visitAggregateOperator(AggregateOperator op, Void arg) throws AlgebricksException {

-        producedVariables.addAll(op.getVariables());

-        return null;

-    }

-

-    @Override

-    public Void visitAssignOperator(AssignOperator op, Void arg) throws AlgebricksException {

-        producedVariables.addAll(op.getVariables());

-        return null;

-    }

-

-    @Override

-    public Void visitDataScanOperator(DataSourceScanOperator op, Void arg) throws AlgebricksException {

-        producedVariables.addAll(op.getVariables());

-        return null;

-    }

-

-    @Override

-    public Void visitDistinctOperator(DistinctOperator op, Void arg) {

-        return null;

-    }

-

-    @Override

-    public Void visitEmptyTupleSourceOperator(EmptyTupleSourceOperator op, Void arg) throws AlgebricksException {

-        return null;

-    }

-

-    @Override

-    public Void visitExchangeOperator(ExchangeOperator op, Void arg) throws AlgebricksException {

-        return null;

-    }

-

-    @Override

-    public Void visitGroupByOperator(GroupByOperator op, Void arg) throws AlgebricksException {

-        for (ILogicalPlan p : op.getNestedPlans()) {

-            for (Mutable<ILogicalOperator> r : p.getRoots()) {

-                VariableUtilities.getLiveVariables(r.getValue(), producedVariables);

-            }

-        }

-        for (Pair<LogicalVariable, Mutable<ILogicalExpression>> p : op.getGroupByList()) {

-            if (p.first != null) {

-                producedVariables.add(p.first);

-            }

-        }

-        for (Pair<LogicalVariable, Mutable<ILogicalExpression>> p : op.getDecorList()) {

-            if (p.first != null) {

-                producedVariables.add(p.first);

-            }

-        }

-        return null;

-    }

-

-    @Override

-    public Void visitInnerJoinOperator(InnerJoinOperator op, Void arg) throws AlgebricksException {

-        return null;

-    }

-

-    @Override

-    public Void visitLeftOuterJoinOperator(LeftOuterJoinOperator op, Void arg) throws AlgebricksException {

-        return null;

-    }

-

-    @Override

-    public Void visitLimitOperator(LimitOperator op, Void arg) throws AlgebricksException {

-        return null;

-    }

-

-    @Override

-    public Void visitNestedTupleSourceOperator(NestedTupleSourceOperator op, Void arg) throws AlgebricksException {

-        return null;

-    }

-

-    @Override

-    public Void visitOrderOperator(OrderOperator op, Void arg) throws AlgebricksException {

-        return null;

-    }

-

-    @Override

-    public Void visitPartitioningSplitOperator(PartitioningSplitOperator op, Void arg) throws AlgebricksException {

-        return null;

-    }

-

-    @Override

-    public Void visitProjectOperator(ProjectOperator op, Void arg) throws AlgebricksException {

-        return null;

-    }

-

-    @Override

-    public Void visitRunningAggregateOperator(RunningAggregateOperator op, Void arg) throws AlgebricksException {

-        producedVariables.addAll(op.getVariables());

-        return null;

-    }

-

-    @Override

-    public Void visitScriptOperator(ScriptOperator op, Void arg) throws AlgebricksException {

-        List<LogicalVariable> usedVariables = new ArrayList<LogicalVariable>();

-        VariableUtilities.getUsedVariables(op, usedVariables);

-        for (LogicalVariable v : op.getOutputVariables()) {

-            if (!usedVariables.contains(v)) {

-                producedVariables.add(v);

-            }

-        }

-        return null;

-    }

-

-    @Override

-    public Void visitSelectOperator(SelectOperator op, Void arg) throws AlgebricksException {

-        return null;

-    }

-

-    @Override

-    public Void visitSubplanOperator(SubplanOperator op, Void arg) throws AlgebricksException {

-        for (ILogicalPlan p : op.getNestedPlans()) {

-            for (Mutable<ILogicalOperator> r : p.getRoots()) {

-                VariableUtilities.getLiveVariables(r.getValue(), producedVariables);

-            }

-        }

-        return null;

-    }

-

-    @Override

-    public Void visitUnionOperator(UnionAllOperator op, Void arg) throws AlgebricksException {

-        for (Triple<LogicalVariable, LogicalVariable, LogicalVariable> t : op.getVariableMappings()) {

-            producedVariables.add(t.third);

-        }

-        return null;

-    }

-

-    @Override

-    public Void visitUnnestMapOperator(UnnestMapOperator op, Void arg) throws AlgebricksException {

-        producedVariables.addAll(op.getVariables());

-        return null;

-    }

-

-    @Override

-    public Void visitUnnestOperator(UnnestOperator op, Void arg) throws AlgebricksException {

-        producedVariables.addAll(op.getVariables());

-        LogicalVariable positionalVariable = op.getPositionalVariable();

-        if (positionalVariable != null) {

-            if (!producedVariables.contains(positionalVariable))

-                producedVariables.add(positionalVariable);

-        }

-        return null;

-    }

-

-    @Override

-    public Void visitWriteOperator(WriteOperator op, Void arg) throws AlgebricksException {

-        return null;

-    }

-

-    @Override

-    public Void visitDistributeResultOperator(DistributeResultOperator op, Void arg) throws AlgebricksException {

-        return null;

-    }

-

-    @Override

-    public Void visitWriteResultOperator(WriteResultOperator op, Void arg) throws AlgebricksException {

-        return null;

-    }

-

-    @Override

-    public Void visitReplicateOperator(ReplicateOperator op, Void arg) throws AlgebricksException {

-        return null;

-    }

-

-    @Override

-    public Void visitInsertDeleteOperator(InsertDeleteOperator op, Void arg) throws AlgebricksException {

-        return null;

-    }

-

-    @Override

-    public Void visitIndexInsertDeleteOperator(IndexInsertDeleteOperator op, Void arg) throws AlgebricksException {

-        return null;

-    }

-

-    @Override

-    public Void visitSinkOperator(SinkOperator op, Void arg) throws AlgebricksException {

-        return null;

-    }

-

-    @Override

-    public Void visitExtensionOperator(ExtensionOperator op, Void arg) throws AlgebricksException {

-        op.getDelegate().getProducedVariables(producedVariables);

-        return null;

-    }

-}

+package edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.visitors;
+
+import java.util.ArrayList;
+import java.util.Collection;
+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.common.utils.Pair;
+import edu.uci.ics.hyracks.algebricks.common.utils.Triple;
+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.LogicalVariable;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AggregateOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AssignOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.DataSourceScanOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.DistinctOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.DistributeResultOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.EmptyTupleSourceOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ExchangeOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ExtensionOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ExternalDataLookupOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.GroupByOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.IndexInsertDeleteOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.InnerJoinOperator;
+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.NestedTupleSourceOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.OrderOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.PartitioningSplitOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ProjectOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ReplicateOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.RunningAggregateOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ScriptOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.SelectOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.SinkOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.SubplanOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.UnionAllOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.UnnestMapOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.UnnestOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.WriteOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.WriteResultOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.visitors.ILogicalOperatorVisitor;
+
+public class ProducedVariableVisitor implements ILogicalOperatorVisitor<Void, Void> {
+
+    private Collection<LogicalVariable> producedVariables;
+
+    public ProducedVariableVisitor(Collection<LogicalVariable> producedVariables) throws AlgebricksException {
+        this.producedVariables = producedVariables;
+    }
+
+    @Override
+    public Void visitAggregateOperator(AggregateOperator op, Void arg) throws AlgebricksException {
+        producedVariables.addAll(op.getVariables());
+        return null;
+    }
+
+    @Override
+    public Void visitAssignOperator(AssignOperator op, Void arg) throws AlgebricksException {
+        producedVariables.addAll(op.getVariables());
+        return null;
+    }
+
+    @Override
+    public Void visitDataScanOperator(DataSourceScanOperator op, Void arg) throws AlgebricksException {
+        producedVariables.addAll(op.getVariables());
+        return null;
+    }
+
+    @Override
+    public Void visitDistinctOperator(DistinctOperator op, Void arg) {
+        return null;
+    }
+
+    @Override
+    public Void visitEmptyTupleSourceOperator(EmptyTupleSourceOperator op, Void arg) throws AlgebricksException {
+        return null;
+    }
+
+    @Override
+    public Void visitExchangeOperator(ExchangeOperator op, Void arg) throws AlgebricksException {
+        return null;
+    }
+
+    @Override
+    public Void visitGroupByOperator(GroupByOperator op, Void arg) throws AlgebricksException {
+        for (ILogicalPlan p : op.getNestedPlans()) {
+            for (Mutable<ILogicalOperator> r : p.getRoots()) {
+                VariableUtilities.getLiveVariables(r.getValue(), producedVariables);
+            }
+        }
+        for (Pair<LogicalVariable, Mutable<ILogicalExpression>> p : op.getGroupByList()) {
+            if (p.first != null) {
+                producedVariables.add(p.first);
+            }
+        }
+        for (Pair<LogicalVariable, Mutable<ILogicalExpression>> p : op.getDecorList()) {
+            if (p.first != null) {
+                producedVariables.add(p.first);
+            }
+        }
+        return null;
+    }
+
+    @Override
+    public Void visitInnerJoinOperator(InnerJoinOperator op, Void arg) throws AlgebricksException {
+        return null;
+    }
+
+    @Override
+    public Void visitLeftOuterJoinOperator(LeftOuterJoinOperator op, Void arg) throws AlgebricksException {
+        return null;
+    }
+
+    @Override
+    public Void visitLimitOperator(LimitOperator op, Void arg) throws AlgebricksException {
+        return null;
+    }
+
+    @Override
+    public Void visitNestedTupleSourceOperator(NestedTupleSourceOperator op, Void arg) throws AlgebricksException {
+        return null;
+    }
+
+    @Override
+    public Void visitOrderOperator(OrderOperator op, Void arg) throws AlgebricksException {
+        return null;
+    }
+
+    @Override
+    public Void visitPartitioningSplitOperator(PartitioningSplitOperator op, Void arg) throws AlgebricksException {
+        return null;
+    }
+
+    @Override
+    public Void visitProjectOperator(ProjectOperator op, Void arg) throws AlgebricksException {
+        return null;
+    }
+
+    @Override
+    public Void visitRunningAggregateOperator(RunningAggregateOperator op, Void arg) throws AlgebricksException {
+        producedVariables.addAll(op.getVariables());
+        return null;
+    }
+
+    @Override
+    public Void visitScriptOperator(ScriptOperator op, Void arg) throws AlgebricksException {
+        List<LogicalVariable> usedVariables = new ArrayList<LogicalVariable>();
+        VariableUtilities.getUsedVariables(op, usedVariables);
+        for (LogicalVariable v : op.getOutputVariables()) {
+            if (!usedVariables.contains(v)) {
+                producedVariables.add(v);
+            }
+        }
+        return null;
+    }
+
+    @Override
+    public Void visitSelectOperator(SelectOperator op, Void arg) throws AlgebricksException {
+        return null;
+    }
+
+    @Override
+    public Void visitSubplanOperator(SubplanOperator op, Void arg) throws AlgebricksException {
+        for (ILogicalPlan p : op.getNestedPlans()) {
+            for (Mutable<ILogicalOperator> r : p.getRoots()) {
+                VariableUtilities.getLiveVariables(r.getValue(), producedVariables);
+            }
+        }
+        return null;
+    }
+
+    @Override
+    public Void visitUnionOperator(UnionAllOperator op, Void arg) throws AlgebricksException {
+        for (Triple<LogicalVariable, LogicalVariable, LogicalVariable> t : op.getVariableMappings()) {
+            producedVariables.add(t.third);
+        }
+        return null;
+    }
+
+    @Override
+    public Void visitUnnestMapOperator(UnnestMapOperator op, Void arg) throws AlgebricksException {
+        producedVariables.addAll(op.getVariables());
+        return null;
+    }
+
+    @Override
+    public Void visitUnnestOperator(UnnestOperator op, Void arg) throws AlgebricksException {
+        producedVariables.addAll(op.getVariables());
+        LogicalVariable positionalVariable = op.getPositionalVariable();
+        if (positionalVariable != null) {
+            if (!producedVariables.contains(positionalVariable))
+                producedVariables.add(positionalVariable);
+        }
+        return null;
+    }
+
+    @Override
+    public Void visitWriteOperator(WriteOperator op, Void arg) throws AlgebricksException {
+        return null;
+    }
+
+    @Override
+    public Void visitDistributeResultOperator(DistributeResultOperator op, Void arg) throws AlgebricksException {
+        return null;
+    }
+
+    @Override
+    public Void visitWriteResultOperator(WriteResultOperator op, Void arg) throws AlgebricksException {
+        return null;
+    }
+
+    @Override
+    public Void visitReplicateOperator(ReplicateOperator op, Void arg) throws AlgebricksException {
+        return null;
+    }
+
+    @Override
+    public Void visitInsertDeleteOperator(InsertDeleteOperator op, Void arg) throws AlgebricksException {
+        return null;
+    }
+
+    @Override
+    public Void visitIndexInsertDeleteOperator(IndexInsertDeleteOperator op, Void arg) throws AlgebricksException {
+        return null;
+    }
+
+    @Override
+    public Void visitSinkOperator(SinkOperator op, Void arg) throws AlgebricksException {
+        return null;
+    }
+
+    @Override
+    public Void visitExtensionOperator(ExtensionOperator op, Void arg) throws AlgebricksException {
+        op.getDelegate().getProducedVariables(producedVariables);
+        return null;
+    }
+
+    @Override
+    public Void visitExternalDataLookupOperator(ExternalDataLookupOperator op, Void arg) throws AlgebricksException {
+        producedVariables.add(op.getVariables().get(0));
+        return null;
+    }
+}
diff --git a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/visitors/SchemaVariableVisitor.java b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/visitors/SchemaVariableVisitor.java
index 5606e72..1d0fd7c 100644
--- a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/visitors/SchemaVariableVisitor.java
+++ b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/visitors/SchemaVariableVisitor.java
@@ -12,280 +12,301 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.visitors;

-

-import java.util.ArrayList;

-import java.util.Collection;

-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.common.utils.Pair;

-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.LogicalVariable;

-import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.VariableReferenceExpression;

-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AggregateOperator;

-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AssignOperator;

-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.DataSourceScanOperator;

-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.DistinctOperator;

-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.DistributeResultOperator;

-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.EmptyTupleSourceOperator;

-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ExchangeOperator;

-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ExtensionOperator;

-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.GroupByOperator;

-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.IndexInsertDeleteOperator;

-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.InnerJoinOperator;

-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.NestedTupleSourceOperator;

-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.OrderOperator;

-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.PartitioningSplitOperator;

-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ProjectOperator;

-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ReplicateOperator;

-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.RunningAggregateOperator;

-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ScriptOperator;

-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.SelectOperator;

-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.SinkOperator;

-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.SubplanOperator;

-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.UnionAllOperator;

-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.UnnestMapOperator;

-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.UnnestOperator;

-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.WriteOperator;

-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.WriteResultOperator;

-import edu.uci.ics.hyracks.algebricks.core.algebra.visitors.ILogicalOperatorVisitor;

-

-public class SchemaVariableVisitor implements ILogicalOperatorVisitor<Void, Void> {

-

-    private Collection<LogicalVariable> schemaVariables;

-

-    public SchemaVariableVisitor(Collection<LogicalVariable> schemaVariables) {

-        this.schemaVariables = schemaVariables;

-    }

-

-    @Override

-    public Void visitAggregateOperator(AggregateOperator op, Void arg) throws AlgebricksException {

-        schemaVariables.addAll(op.getVariables());

-        return null;

-    }

-

-    @Override

-    public Void visitAssignOperator(AssignOperator op, Void arg) throws AlgebricksException {

-        standardLayout(op);

-        return null;

-    }

-

-    @Override

-    public Void visitDataScanOperator(DataSourceScanOperator op, Void arg) throws AlgebricksException {

-        standardLayout(op);

-        return null;

-    }

-

-    @Override

-    public Void visitDistinctOperator(DistinctOperator op, Void arg) throws AlgebricksException {

-        List<LogicalVariable> allLiveVars = new ArrayList<LogicalVariable>();

-        for (Mutable<ILogicalOperator> c : op.getInputs()) {

-            VariableUtilities.getLiveVariables(c.getValue(), allLiveVars);

-        }

-        VariableUtilities.getProducedVariables(op, allLiveVars);

-        /** put distinct vars first */

-        schemaVariables.addAll(op.getDistinctByVarList());

-        /** then other live vars */

-        for (LogicalVariable var : allLiveVars) {

-            if (!schemaVariables.contains(var)) {

-                schemaVariables.add(var);

-            }

-        }

-        return null;

-    }

-

-    @Override

-    public Void visitEmptyTupleSourceOperator(EmptyTupleSourceOperator op, Void arg) throws AlgebricksException {

-        standardLayout(op);

-        return null;

-    }

-

-    @Override

-    public Void visitExchangeOperator(ExchangeOperator op, Void arg) throws AlgebricksException {

-        standardLayout(op);

-        return null;

-    }

-

-    @Override

-    public Void visitGroupByOperator(GroupByOperator op, Void arg) throws AlgebricksException {

-        for (ILogicalPlan p : op.getNestedPlans()) {

-            for (Mutable<ILogicalOperator> r : p.getRoots()) {

-                VariableUtilities.getLiveVariables(r.getValue(), schemaVariables);

-            }

-        }

-        for (Pair<LogicalVariable, Mutable<ILogicalExpression>> p : op.getGroupByList()) {

-            if (p.first != null) {

-                schemaVariables.add(p.first);

-            }

-        }

-        for (Pair<LogicalVariable, Mutable<ILogicalExpression>> p : op.getDecorList()) {

-            if (p.first != null) {

-                schemaVariables.add(p.first);

-            } else {

-                ILogicalExpression e = p.second.getValue();

-                if (e.getExpressionTag() == LogicalExpressionTag.VARIABLE) {

-                    schemaVariables.add(((VariableReferenceExpression) e).getVariableReference());

-                }

-            }

-        }

-        return null;

-    }

-

-    @Override

-    public Void visitInnerJoinOperator(InnerJoinOperator op, Void arg) throws AlgebricksException {

-        standardLayout(op);

-        return null;

-    }

-

-    @Override

-    public Void visitLeftOuterJoinOperator(LeftOuterJoinOperator op, Void arg) throws AlgebricksException {

-        standardLayout(op);

-        return null;

-    }

-

-    @Override

-    public Void visitLimitOperator(LimitOperator op, Void arg) throws AlgebricksException {

-        standardLayout(op);

-        return null;

-    }

-

-    @Override

-    public Void visitNestedTupleSourceOperator(NestedTupleSourceOperator op, Void arg) throws AlgebricksException {

-        VariableUtilities.getLiveVariables(op.getSourceOperator(), schemaVariables);

-        return null;

-    }

-

-    @Override

-    public Void visitOrderOperator(OrderOperator op, Void arg) throws AlgebricksException {

-        standardLayout(op);

-        return null;

-    }

-

-    @Override

-    public Void visitPartitioningSplitOperator(PartitioningSplitOperator op, Void arg) throws AlgebricksException {

-        standardLayout(op);

-        return null;

-    }

-

-    @Override

-    public Void visitProjectOperator(ProjectOperator op, Void arg) throws AlgebricksException {

-        schemaVariables.addAll(op.getVariables());

-        return null;

-    }

-

-    @Override

-    public Void visitRunningAggregateOperator(RunningAggregateOperator op, Void arg) throws AlgebricksException {

-        // VariableUtilities.getProducedVariables(op, schemaVariables);

-        standardLayout(op);

-        return null;

-    }

-

-    @Override

-    public Void visitScriptOperator(ScriptOperator op, Void arg) throws AlgebricksException {

-        schemaVariables.addAll(op.getOutputVariables());

-        return null;

-    }

-

-    @Override

-    public Void visitSelectOperator(SelectOperator op, Void arg) throws AlgebricksException {

-        standardLayout(op);

-        return null;

-    }

-

-    @Override

-    public Void visitSubplanOperator(SubplanOperator op, Void arg) throws AlgebricksException {

-        for (Mutable<ILogicalOperator> c : op.getInputs()) {

-            VariableUtilities.getLiveVariables(c.getValue(), schemaVariables);

-        }

-        VariableUtilities.getProducedVariables(op, schemaVariables);

-        for (ILogicalPlan p : op.getNestedPlans()) {

-            for (Mutable<ILogicalOperator> r : p.getRoots()) {

-                VariableUtilities.getLiveVariables(r.getValue(), schemaVariables);

-            }

-        }

-        return null;

-    }

-

-    @Override

-    public Void visitUnionOperator(UnionAllOperator op, Void arg) throws AlgebricksException {

-        VariableUtilities.getProducedVariables(op, schemaVariables);

-        return null;

-    }

-

-    @Override

-    public Void visitUnnestMapOperator(UnnestMapOperator op, Void arg) throws AlgebricksException {

-        standardLayout(op);

-        return null;

-    }

-

-    @Override

-    public Void visitUnnestOperator(UnnestOperator op, Void arg) throws AlgebricksException {

-        standardLayout(op);

-        return null;

-    }

-

-    @Override

-    public Void visitWriteOperator(WriteOperator op, Void arg) throws AlgebricksException {

-        standardLayout(op);

-        return null;

-    }

-

-    @Override

-    public Void visitDistributeResultOperator(DistributeResultOperator op, Void arg) throws AlgebricksException {

-        standardLayout(op);

-        return null;

-    }

-

-    @Override

-    public Void visitWriteResultOperator(WriteResultOperator op, Void arg) throws AlgebricksException {

-        standardLayout(op);

-        return null;

-    }

-

-    private void standardLayout(ILogicalOperator op) throws AlgebricksException {

-        for (Mutable<ILogicalOperator> c : op.getInputs()) {

-            VariableUtilities.getLiveVariables(c.getValue(), schemaVariables);

-        }

-        VariableUtilities.getProducedVariables(op, schemaVariables);

-    }

-

-    @Override

-    public Void visitReplicateOperator(ReplicateOperator op, Void arg) throws AlgebricksException {

-        standardLayout(op);

-        return null;

-    }

-

-    @Override

-    public Void visitInsertDeleteOperator(InsertDeleteOperator op, Void arg) throws AlgebricksException {

-        standardLayout(op);

-        return null;

-    }

-

-    @Override

-    public Void visitIndexInsertDeleteOperator(IndexInsertDeleteOperator op, Void arg) throws AlgebricksException {

-        standardLayout(op);

-        return null;

-    }

-

-    @Override

-    public Void visitSinkOperator(SinkOperator op, Void arg) throws AlgebricksException {

-        standardLayout(op);

-        return null;

-    }

-

-    @Override

-    public Void visitExtensionOperator(ExtensionOperator op, Void arg) throws AlgebricksException {

-        standardLayout(op);

-        return null;

-    }

-

-}

+package edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.visitors;
+
+import java.util.ArrayList;
+import java.util.Collection;
+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.common.utils.Pair;
+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.LogicalVariable;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.VariableReferenceExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AggregateOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AssignOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.DataSourceScanOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.DistinctOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.DistributeResultOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.EmptyTupleSourceOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ExchangeOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ExtensionOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ExternalDataLookupOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.GroupByOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.IndexInsertDeleteOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.InnerJoinOperator;
+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.NestedTupleSourceOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.OrderOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.PartitioningSplitOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ProjectOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ReplicateOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.RunningAggregateOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ScriptOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.SelectOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.SinkOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.SubplanOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.UnionAllOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.UnnestMapOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.UnnestOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.WriteOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.WriteResultOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.visitors.ILogicalOperatorVisitor;
+
+public class SchemaVariableVisitor implements ILogicalOperatorVisitor<Void, Void> {
+
+    private Collection<LogicalVariable> schemaVariables;
+
+    public SchemaVariableVisitor(Collection<LogicalVariable> schemaVariables) {
+        this.schemaVariables = schemaVariables;
+    }
+
+    @Override
+    public Void visitAggregateOperator(AggregateOperator op, Void arg) throws AlgebricksException {
+        schemaVariables.addAll(op.getVariables());
+        return null;
+    }
+
+    @Override
+    public Void visitAssignOperator(AssignOperator op, Void arg) throws AlgebricksException {
+        standardLayout(op);
+        return null;
+    }
+
+    @Override
+    public Void visitDataScanOperator(DataSourceScanOperator op, Void arg) throws AlgebricksException {
+        standardLayout(op);
+        return null;
+    }
+
+    @Override
+    public Void visitDistinctOperator(DistinctOperator op, Void arg) throws AlgebricksException {
+        List<LogicalVariable> allLiveVars = new ArrayList<LogicalVariable>();
+        for (Mutable<ILogicalOperator> c : op.getInputs()) {
+            VariableUtilities.getLiveVariables(c.getValue(), allLiveVars);
+        }
+        VariableUtilities.getProducedVariables(op, allLiveVars);
+        /** put distinct vars first */
+        schemaVariables.addAll(op.getDistinctByVarList());
+        /** then other live vars */
+        for (LogicalVariable var : allLiveVars) {
+            if (!schemaVariables.contains(var)) {
+                schemaVariables.add(var);
+            }
+        }
+        return null;
+    }
+
+    @Override
+    public Void visitEmptyTupleSourceOperator(EmptyTupleSourceOperator op, Void arg) throws AlgebricksException {
+        standardLayout(op);
+        return null;
+    }
+
+    @Override
+    public Void visitExchangeOperator(ExchangeOperator op, Void arg) throws AlgebricksException {
+        standardLayout(op);
+        return null;
+    }
+
+    @Override
+    public Void visitGroupByOperator(GroupByOperator op, Void arg) throws AlgebricksException {
+        for (ILogicalPlan p : op.getNestedPlans()) {
+            for (Mutable<ILogicalOperator> r : p.getRoots()) {
+                VariableUtilities.getLiveVariables(r.getValue(), schemaVariables);
+            }
+        }
+        for (Pair<LogicalVariable, Mutable<ILogicalExpression>> p : op.getGroupByList()) {
+            if (p.first != null) {
+                schemaVariables.add(p.first);
+            }
+        }
+        for (Pair<LogicalVariable, Mutable<ILogicalExpression>> p : op.getDecorList()) {
+            if (p.first != null) {
+                schemaVariables.add(p.first);
+            } else {
+                ILogicalExpression e = p.second.getValue();
+                if (e.getExpressionTag() == LogicalExpressionTag.VARIABLE) {
+                    schemaVariables.add(((VariableReferenceExpression) e).getVariableReference());
+                }
+            }
+        }
+        return null;
+    }
+
+    @Override
+    public Void visitInnerJoinOperator(InnerJoinOperator op, Void arg) throws AlgebricksException {
+        standardLayout(op);
+        return null;
+    }
+
+    @Override
+    public Void visitLeftOuterJoinOperator(LeftOuterJoinOperator op, Void arg) throws AlgebricksException {
+        standardLayout(op);
+        return null;
+    }
+
+    @Override
+    public Void visitLimitOperator(LimitOperator op, Void arg) throws AlgebricksException {
+        standardLayout(op);
+        return null;
+    }
+
+    @Override
+    public Void visitNestedTupleSourceOperator(NestedTupleSourceOperator op, Void arg) throws AlgebricksException {
+        VariableUtilities.getLiveVariables(op.getSourceOperator(), schemaVariables);
+        return null;
+    }
+
+    @Override
+    public Void visitOrderOperator(OrderOperator op, Void arg) throws AlgebricksException {
+        standardLayout(op);
+        return null;
+    }
+
+    @Override
+    public Void visitPartitioningSplitOperator(PartitioningSplitOperator op, Void arg) throws AlgebricksException {
+        standardLayout(op);
+        return null;
+    }
+
+    @Override
+    public Void visitProjectOperator(ProjectOperator op, Void arg) throws AlgebricksException {
+        schemaVariables.addAll(op.getVariables());
+        return null;
+    }
+
+    @Override
+    public Void visitRunningAggregateOperator(RunningAggregateOperator op, Void arg) throws AlgebricksException {
+        // VariableUtilities.getProducedVariables(op, schemaVariables);
+        standardLayout(op);
+        return null;
+    }
+
+    @Override
+    public Void visitScriptOperator(ScriptOperator op, Void arg) throws AlgebricksException {
+        schemaVariables.addAll(op.getOutputVariables());
+        return null;
+    }
+
+    @Override
+    public Void visitSelectOperator(SelectOperator op, Void arg) throws AlgebricksException {
+        standardLayout(op);
+        return null;
+    }
+
+    @Override
+    public Void visitSubplanOperator(SubplanOperator op, Void arg) throws AlgebricksException {
+        for (Mutable<ILogicalOperator> c : op.getInputs()) {
+            VariableUtilities.getLiveVariables(c.getValue(), schemaVariables);
+        }
+        VariableUtilities.getProducedVariables(op, schemaVariables);
+        for (ILogicalPlan p : op.getNestedPlans()) {
+            for (Mutable<ILogicalOperator> r : p.getRoots()) {
+                VariableUtilities.getLiveVariables(r.getValue(), schemaVariables);
+            }
+        }
+        return null;
+    }
+
+    @Override
+    public Void visitUnionOperator(UnionAllOperator op, Void arg) throws AlgebricksException {
+        VariableUtilities.getProducedVariables(op, schemaVariables);
+        return null;
+    }
+
+    @Override
+    public Void visitUnnestMapOperator(UnnestMapOperator op, Void arg) throws AlgebricksException {
+        standardLayout(op);
+        return null;
+    }
+
+    @Override
+    public Void visitUnnestOperator(UnnestOperator op, Void arg) throws AlgebricksException {
+        standardLayout(op);
+        return null;
+    }
+
+    @Override
+    public Void visitWriteOperator(WriteOperator op, Void arg) throws AlgebricksException {
+        standardLayout(op);
+        return null;
+    }
+
+    @Override
+    public Void visitDistributeResultOperator(DistributeResultOperator op, Void arg) throws AlgebricksException {
+        standardLayout(op);
+        return null;
+    }
+
+    @Override
+    public Void visitWriteResultOperator(WriteResultOperator op, Void arg) throws AlgebricksException {
+        standardLayout(op);
+        return null;
+    }
+
+    private void standardLayout(ILogicalOperator op) throws AlgebricksException {
+        for (Mutable<ILogicalOperator> c : op.getInputs()) {
+            VariableUtilities.getLiveVariables(c.getValue(), schemaVariables);
+        }
+        VariableUtilities.getProducedVariables(op, schemaVariables);
+    }
+
+    @Override
+    public Void visitReplicateOperator(ReplicateOperator op, Void arg) throws AlgebricksException {
+        standardLayout(op);
+        return null;
+    }
+
+    @Override
+    public Void visitInsertDeleteOperator(InsertDeleteOperator op, Void arg) throws AlgebricksException {
+        standardLayout(op);
+        return null;
+    }
+
+    @Override
+    public Void visitIndexInsertDeleteOperator(IndexInsertDeleteOperator op, Void arg) throws AlgebricksException {
+        standardLayout(op);
+        return null;
+    }
+
+    @Override
+    public Void visitSinkOperator(SinkOperator op, Void arg) throws AlgebricksException {
+        standardLayout(op);
+        return null;
+    }
+
+    @Override
+    public Void visitExtensionOperator(ExtensionOperator op, Void arg) throws AlgebricksException {
+        standardLayout(op);
+        return null;
+    }
+
+    @Override
+    public Void visitExternalDataLookupOperator(ExternalDataLookupOperator op, Void arg) throws AlgebricksException {
+        ArrayList<LogicalVariable> liveVariables = new ArrayList<LogicalVariable>();
+        ArrayList<LogicalVariable> usedVariables = new ArrayList<LogicalVariable>();
+        //get used variables
+        op.getExpressionRef().getValue().getUsedVariables(usedVariables);
+        //live variables - used variables
+        for (Mutable<ILogicalOperator> c : op.getInputs()) {
+            VariableUtilities.getLiveVariables(c.getValue(), liveVariables);
+        }
+        for (LogicalVariable v : liveVariables) {
+            if (!usedVariables.contains(v)) {
+                schemaVariables.add(v);
+            }
+        }
+        VariableUtilities.getProducedVariables(op, schemaVariables);
+        //+ produced variables
+        return null;
+    }
+
+}
diff --git a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/visitors/SubstituteVariableVisitor.java b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/visitors/SubstituteVariableVisitor.java
index 3d6b989..729a996 100644
--- a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/visitors/SubstituteVariableVisitor.java
+++ b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/visitors/SubstituteVariableVisitor.java
@@ -12,410 +12,426 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.visitors;

-

-import java.util.ArrayList;

-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.common.utils.Pair;

-import edu.uci.ics.hyracks.algebricks.common.utils.Triple;

-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.LogicalVariable;

-import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;

-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;

-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AggregateOperator;

-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AssignOperator;

-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.DataSourceScanOperator;

-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.DistinctOperator;

-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.DistributeResultOperator;

-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.EmptyTupleSourceOperator;

-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ExchangeOperator;

-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ExtensionOperator;

-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.GroupByOperator;

-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.IndexInsertDeleteOperator;

-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.InnerJoinOperator;

-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.NestedTupleSourceOperator;

-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.OrderOperator;

-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.OrderOperator.IOrder;

-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.PartitioningSplitOperator;

-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ProjectOperator;

-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ReplicateOperator;

-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.RunningAggregateOperator;

-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ScriptOperator;

-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.SelectOperator;

-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.SinkOperator;

-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.SubplanOperator;

-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.UnionAllOperator;

-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.UnnestMapOperator;

-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.UnnestOperator;

-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.WriteOperator;

-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.WriteResultOperator;

-import edu.uci.ics.hyracks.algebricks.core.algebra.typing.ITypingContext;

-import edu.uci.ics.hyracks.algebricks.core.algebra.util.OperatorManipulationUtil;

-import edu.uci.ics.hyracks.algebricks.core.algebra.visitors.ILogicalOperatorVisitor;

-

-public class SubstituteVariableVisitor implements ILogicalOperatorVisitor<Void, Pair<LogicalVariable, LogicalVariable>> {

-

-    private final boolean goThroughNts;

-    private final ITypingContext ctx;

-

-    public SubstituteVariableVisitor(boolean goThroughNts, ITypingContext ctx) {

-        this.goThroughNts = goThroughNts;

-        this.ctx = ctx;

-    }

-

-    @Override

-    public Void visitAggregateOperator(AggregateOperator op, Pair<LogicalVariable, LogicalVariable> pair)

-            throws AlgebricksException {

-        List<LogicalVariable> variables = op.getVariables();

-        int n = variables.size();

-        for (int i = 0; i < n; i++) {

-            if (variables.get(i).equals(pair.first)) {

-                variables.set(i, pair.second);

-            } else {

-                op.getExpressions().get(i).getValue().substituteVar(pair.first, pair.second);

-            }

-        }

-        substVarTypes(op, pair);

-        return null;

-    }

-

-    @Override

-    public Void visitAssignOperator(AssignOperator op, Pair<LogicalVariable, LogicalVariable> pair)

-            throws AlgebricksException {

-        List<LogicalVariable> variables = op.getVariables();

-        int n = variables.size();

-        for (int i = 0; i < n; i++) {

-            if (variables.get(i).equals(pair.first)) {

-                variables.set(i, pair.second);

-            } else {

-                op.getExpressions().get(i).getValue().substituteVar(pair.first, pair.second);

-            }

-        }

-        substVarTypes(op, pair);

-        return null;

-    }

-

-    @Override

-    public Void visitDataScanOperator(DataSourceScanOperator op, Pair<LogicalVariable, LogicalVariable> pair)

-            throws AlgebricksException {

-        List<LogicalVariable> variables = op.getVariables();

-        for (int i = 0; i < variables.size(); i++) {

-            if (variables.get(i) == pair.first) {

-                variables.set(i, pair.second);

-                return null;

-            }

-        }

-        substVarTypes(op, pair);

-        return null;

-    }

-

-    @Override

-    public Void visitDistinctOperator(DistinctOperator op, Pair<LogicalVariable, LogicalVariable> pair)

-            throws AlgebricksException {

-        for (Mutable<ILogicalExpression> eRef : op.getExpressions()) {

-            eRef.getValue().substituteVar(pair.first, pair.second);

-        }

-        substVarTypes(op, pair);

-        return null;

-    }

-

-    @Override

-    public Void visitEmptyTupleSourceOperator(EmptyTupleSourceOperator op, Pair<LogicalVariable, LogicalVariable> pair) {

-        // does not use any variable

-        return null;

-    }

-

-    @Override

-    public Void visitExchangeOperator(ExchangeOperator op, Pair<LogicalVariable, LogicalVariable> pair) {

-        // does not use any variable

-        return null;

-    }

-

-    @Override

-    public Void visitGroupByOperator(GroupByOperator op, Pair<LogicalVariable, LogicalVariable> pair)

-            throws AlgebricksException {

-        subst(pair.first, pair.second, op.getGroupByList());

-        subst(pair.first, pair.second, op.getDecorList());

-        for (ILogicalPlan p : op.getNestedPlans()) {

-            for (Mutable<ILogicalOperator> r : p.getRoots()) {

-                OperatorManipulationUtil.substituteVarRec((AbstractLogicalOperator) r.getValue(), pair.first,

-                        pair.second, goThroughNts, ctx);

-            }

-        }

-        substVarTypes(op, pair);

-        return null;

-    }

-

-    @Override

-    public Void visitInnerJoinOperator(InnerJoinOperator op, Pair<LogicalVariable, LogicalVariable> pair)

-            throws AlgebricksException {

-        op.getCondition().getValue().substituteVar(pair.first, pair.second);

-        substVarTypes(op, pair);

-        return null;

-    }

-

-    @Override

-    public Void visitLeftOuterJoinOperator(LeftOuterJoinOperator op, Pair<LogicalVariable, LogicalVariable> pair)

-            throws AlgebricksException {

-        op.getCondition().getValue().substituteVar(pair.first, pair.second);

-        substVarTypes(op, pair);

-        return null;

-    }

-

-    @Override

-    public Void visitLimitOperator(LimitOperator op, Pair<LogicalVariable, LogicalVariable> pair)

-            throws AlgebricksException {

-        op.getMaxObjects().getValue().substituteVar(pair.first, pair.second);

-        ILogicalExpression offset = op.getOffset().getValue();

-        if (offset != null) {

-            offset.substituteVar(pair.first, pair.second);

-        }

-        substVarTypes(op, pair);

-        return null;

-    }

-

-    @Override

-    public Void visitNestedTupleSourceOperator(NestedTupleSourceOperator op, Pair<LogicalVariable, LogicalVariable> pair)

-            throws AlgebricksException {

-        return null;

-    }

-

-    @Override

-    public Void visitOrderOperator(OrderOperator op, Pair<LogicalVariable, LogicalVariable> pair)

-            throws AlgebricksException {

-        for (Pair<IOrder, Mutable<ILogicalExpression>> oe : op.getOrderExpressions()) {

-            oe.second.getValue().substituteVar(pair.first, pair.second);

-        }

-        substVarTypes(op, pair);

-        return null;

-    }

-

-    @Override

-    public Void visitPartitioningSplitOperator(PartitioningSplitOperator op, Pair<LogicalVariable, LogicalVariable> pair)

-            throws AlgebricksException {

-        for (Mutable<ILogicalExpression> e : op.getExpressions()) {

-            e.getValue().substituteVar(pair.first, pair.second);

-        }

-        substVarTypes(op, pair);

-        return null;

-    }

-

-    @Override

-    public Void visitProjectOperator(ProjectOperator op, Pair<LogicalVariable, LogicalVariable> pair)

-            throws AlgebricksException {

-        List<LogicalVariable> usedVariables = op.getVariables();

-        int n = usedVariables.size();

-        for (int i = 0; i < n; i++) {

-            LogicalVariable v = usedVariables.get(i);

-            if (v.equals(pair.first)) {

-                usedVariables.set(i, pair.second);

-            }

-        }

-        substVarTypes(op, pair);

-        return null;

-    }

-

-    @Override

-    public Void visitRunningAggregateOperator(RunningAggregateOperator op, Pair<LogicalVariable, LogicalVariable> pair)

-            throws AlgebricksException {

-        List<LogicalVariable> variables = op.getVariables();

-        int n = variables.size();

-        for (int i = 0; i < n; i++) {

-            if (variables.get(i).equals(pair.first)) {

-                variables.set(i, pair.second);

-            } else {

-                op.getExpressions().get(i).getValue().substituteVar(pair.first, pair.second);

-            }

-        }

-        substVarTypes(op, pair);

-        return null;

-    }

-

-    @Override

-    public Void visitScriptOperator(ScriptOperator op, Pair<LogicalVariable, LogicalVariable> pair)

-            throws AlgebricksException {

-        substInArray(op.getInputVariables(), pair.first, pair.second);

-        substInArray(op.getOutputVariables(), pair.first, pair.second);

-        substVarTypes(op, pair);

-        return null;

-    }

-

-    @Override

-    public Void visitSelectOperator(SelectOperator op, Pair<LogicalVariable, LogicalVariable> pair) {

-        op.getCondition().getValue().substituteVar(pair.first, pair.second);

-        return null;

-    }

-

-    @Override

-    public Void visitSubplanOperator(SubplanOperator op, Pair<LogicalVariable, LogicalVariable> pair)

-            throws AlgebricksException {

-        for (ILogicalPlan p : op.getNestedPlans()) {

-            for (Mutable<ILogicalOperator> r : p.getRoots()) {

-                OperatorManipulationUtil.substituteVarRec((AbstractLogicalOperator) r.getValue(), pair.first,

-                        pair.second, goThroughNts, ctx);

-            }

-        }

-        return null;

-    }

-

-    @Override

-    public Void visitUnionOperator(UnionAllOperator op, Pair<LogicalVariable, LogicalVariable> pair)

-            throws AlgebricksException {

-        List<Triple<LogicalVariable, LogicalVariable, LogicalVariable>> varMap = op.getVariableMappings();

-        for (Triple<LogicalVariable, LogicalVariable, LogicalVariable> t : varMap) {

-            if (t.first.equals(pair.first)) {

-                t.first = pair.second;

-            }

-            if (t.second.equals(pair.first)) {

-                t.second = pair.second;

-            }

-            if (t.third.equals(pair.first)) {

-                t.third = pair.second;

-            }

-        }

-        substVarTypes(op, pair);

-        return null;

-    }

-

-    @Override

-    public Void visitUnnestMapOperator(UnnestMapOperator op, Pair<LogicalVariable, LogicalVariable> pair)

-            throws AlgebricksException {

-        List<LogicalVariable> variables = op.getVariables();

-        for (int i = 0; i < variables.size(); i++) {

-            if (variables.get(i) == pair.first) {

-                variables.set(i, pair.second);

-                return null;

-            }

-        }

-        op.getExpressionRef().getValue().substituteVar(pair.first, pair.second);

-        substVarTypes(op, pair);

-        return null;

-    }

-

-    @Override

-    public Void visitUnnestOperator(UnnestOperator op, Pair<LogicalVariable, LogicalVariable> pair)

-            throws AlgebricksException {

-        List<LogicalVariable> variables = op.getVariables();

-        for (int i = 0; i < variables.size(); i++) {

-            if (variables.get(i) == pair.first) {

-                variables.set(i, pair.second);

-                return null;

-            }

-        }

-        op.getExpressionRef().getValue().substituteVar(pair.first, pair.second);

-        substVarTypes(op, pair);

-        return null;

-    }

-

-    @Override

-    public Void visitWriteOperator(WriteOperator op, Pair<LogicalVariable, LogicalVariable> pair)

-            throws AlgebricksException {

-        for (Mutable<ILogicalExpression> e : op.getExpressions()) {

-            e.getValue().substituteVar(pair.first, pair.second);

-        }

-        substVarTypes(op, pair);

-        return null;

-    }

-

-    @Override

-    public Void visitDistributeResultOperator(DistributeResultOperator op, Pair<LogicalVariable, LogicalVariable> pair)

-            throws AlgebricksException {

-        for (Mutable<ILogicalExpression> e : op.getExpressions()) {

-            e.getValue().substituteVar(pair.first, pair.second);

-        }

-        substVarTypes(op, pair);

-        return null;

-    }

-

-    @Override

-    public Void visitWriteResultOperator(WriteResultOperator op, Pair<LogicalVariable, LogicalVariable> pair)

-            throws AlgebricksException {

-        op.getPayloadExpression().getValue().substituteVar(pair.first, pair.second);

-        for (Mutable<ILogicalExpression> e : op.getKeyExpressions()) {

-            e.getValue().substituteVar(pair.first, pair.second);

-        }

-        substVarTypes(op, pair);

-        return null;

-    }

-

-    private void subst(LogicalVariable v1, LogicalVariable v2,

-            List<Pair<LogicalVariable, Mutable<ILogicalExpression>>> varExprPairList) {

-        for (Pair<LogicalVariable, Mutable<ILogicalExpression>> ve : varExprPairList) {

-            if (ve.first != null && ve.first.equals(v1)) {

-                ve.first = v2;

-                return;

-            }

-            ve.second.getValue().substituteVar(v1, v2);

-        }

-    }

-

-    private void substInArray(ArrayList<LogicalVariable> varArray, LogicalVariable v1, LogicalVariable v2) {

-        for (int i = 0; i < varArray.size(); i++) {

-            LogicalVariable v = varArray.get(i);

-            if (v == v1) {

-                varArray.set(i, v2);

-            }

-        }

-    }

-

-    @Override

-    public Void visitReplicateOperator(ReplicateOperator op, Pair<LogicalVariable, LogicalVariable> arg)

-            throws AlgebricksException {

-        op.substituteVar(arg.first, arg.second);

-        return null;

-    }

-

-    @Override

-    public Void visitInsertDeleteOperator(InsertDeleteOperator op, Pair<LogicalVariable, LogicalVariable> pair)

-            throws AlgebricksException {

-        op.getPayloadExpression().getValue().substituteVar(pair.first, pair.second);

-        for (Mutable<ILogicalExpression> e : op.getPrimaryKeyExpressions()) {

-            e.getValue().substituteVar(pair.first, pair.second);

-        }

-        substVarTypes(op, pair);

-        return null;

-    }

-

-    @Override

-    public Void visitIndexInsertDeleteOperator(IndexInsertDeleteOperator op, Pair<LogicalVariable, LogicalVariable> pair)

-            throws AlgebricksException {

-        for (Mutable<ILogicalExpression> e : op.getPrimaryKeyExpressions()) {

-            e.getValue().substituteVar(pair.first, pair.second);

-        }

-        for (Mutable<ILogicalExpression> e : op.getSecondaryKeyExpressions()) {

-            e.getValue().substituteVar(pair.first, pair.second);

-        }

-        substVarTypes(op, pair);

-        return null;

-    }

-

-    @Override

-    public Void visitSinkOperator(SinkOperator op, Pair<LogicalVariable, LogicalVariable> pair)

-            throws AlgebricksException {

-        return null;

-    }

-

-    private void substVarTypes(ILogicalOperator op, Pair<LogicalVariable, LogicalVariable> arg)

-            throws AlgebricksException {

-        if (ctx == null) {

-            return;

-        }

-        IVariableTypeEnvironment env = ctx.getOutputTypeEnvironment(op);

-        env.substituteProducedVariable(arg.first, arg.second);

-    }

-

-    @Override

-    public Void visitExtensionOperator(ExtensionOperator op, Pair<LogicalVariable, LogicalVariable> arg)

-            throws AlgebricksException {

-        return null;

-    }

-}

+package edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.visitors;
+
+import java.util.ArrayList;
+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.common.utils.Pair;
+import edu.uci.ics.hyracks.algebricks.common.utils.Triple;
+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.LogicalVariable;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AggregateOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AssignOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.DataSourceScanOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.DistinctOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.DistributeResultOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.EmptyTupleSourceOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ExchangeOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ExtensionOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ExternalDataLookupOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.GroupByOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.IndexInsertDeleteOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.InnerJoinOperator;
+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.NestedTupleSourceOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.OrderOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.OrderOperator.IOrder;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.PartitioningSplitOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ProjectOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ReplicateOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.RunningAggregateOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ScriptOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.SelectOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.SinkOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.SubplanOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.UnionAllOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.UnnestMapOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.UnnestOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.WriteOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.WriteResultOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.typing.ITypingContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.util.OperatorManipulationUtil;
+import edu.uci.ics.hyracks.algebricks.core.algebra.visitors.ILogicalOperatorVisitor;
+
+public class SubstituteVariableVisitor implements ILogicalOperatorVisitor<Void, Pair<LogicalVariable, LogicalVariable>> {
+
+    private final boolean goThroughNts;
+    private final ITypingContext ctx;
+
+    public SubstituteVariableVisitor(boolean goThroughNts, ITypingContext ctx) {
+        this.goThroughNts = goThroughNts;
+        this.ctx = ctx;
+    }
+
+    @Override
+    public Void visitAggregateOperator(AggregateOperator op, Pair<LogicalVariable, LogicalVariable> pair)
+            throws AlgebricksException {
+        List<LogicalVariable> variables = op.getVariables();
+        int n = variables.size();
+        for (int i = 0; i < n; i++) {
+            if (variables.get(i).equals(pair.first)) {
+                variables.set(i, pair.second);
+            } else {
+                op.getExpressions().get(i).getValue().substituteVar(pair.first, pair.second);
+            }
+        }
+        substVarTypes(op, pair);
+        return null;
+    }
+
+    @Override
+    public Void visitAssignOperator(AssignOperator op, Pair<LogicalVariable, LogicalVariable> pair)
+            throws AlgebricksException {
+        List<LogicalVariable> variables = op.getVariables();
+        int n = variables.size();
+        for (int i = 0; i < n; i++) {
+            if (variables.get(i).equals(pair.first)) {
+                variables.set(i, pair.second);
+            } else {
+                op.getExpressions().get(i).getValue().substituteVar(pair.first, pair.second);
+            }
+        }
+        substVarTypes(op, pair);
+        return null;
+    }
+
+    @Override
+    public Void visitDataScanOperator(DataSourceScanOperator op, Pair<LogicalVariable, LogicalVariable> pair)
+            throws AlgebricksException {
+        List<LogicalVariable> variables = op.getVariables();
+        for (int i = 0; i < variables.size(); i++) {
+            if (variables.get(i) == pair.first) {
+                variables.set(i, pair.second);
+                return null;
+            }
+        }
+        substVarTypes(op, pair);
+        return null;
+    }
+
+    @Override
+    public Void visitDistinctOperator(DistinctOperator op, Pair<LogicalVariable, LogicalVariable> pair)
+            throws AlgebricksException {
+        for (Mutable<ILogicalExpression> eRef : op.getExpressions()) {
+            eRef.getValue().substituteVar(pair.first, pair.second);
+        }
+        substVarTypes(op, pair);
+        return null;
+    }
+
+    @Override
+    public Void visitEmptyTupleSourceOperator(EmptyTupleSourceOperator op, Pair<LogicalVariable, LogicalVariable> pair) {
+        // does not use any variable
+        return null;
+    }
+
+    @Override
+    public Void visitExchangeOperator(ExchangeOperator op, Pair<LogicalVariable, LogicalVariable> pair) {
+        // does not use any variable
+        return null;
+    }
+
+    @Override
+    public Void visitGroupByOperator(GroupByOperator op, Pair<LogicalVariable, LogicalVariable> pair)
+            throws AlgebricksException {
+        subst(pair.first, pair.second, op.getGroupByList());
+        subst(pair.first, pair.second, op.getDecorList());
+        for (ILogicalPlan p : op.getNestedPlans()) {
+            for (Mutable<ILogicalOperator> r : p.getRoots()) {
+                OperatorManipulationUtil.substituteVarRec((AbstractLogicalOperator) r.getValue(), pair.first,
+                        pair.second, goThroughNts, ctx);
+            }
+        }
+        substVarTypes(op, pair);
+        return null;
+    }
+
+    @Override
+    public Void visitInnerJoinOperator(InnerJoinOperator op, Pair<LogicalVariable, LogicalVariable> pair)
+            throws AlgebricksException {
+        op.getCondition().getValue().substituteVar(pair.first, pair.second);
+        substVarTypes(op, pair);
+        return null;
+    }
+
+    @Override
+    public Void visitLeftOuterJoinOperator(LeftOuterJoinOperator op, Pair<LogicalVariable, LogicalVariable> pair)
+            throws AlgebricksException {
+        op.getCondition().getValue().substituteVar(pair.first, pair.second);
+        substVarTypes(op, pair);
+        return null;
+    }
+
+    @Override
+    public Void visitLimitOperator(LimitOperator op, Pair<LogicalVariable, LogicalVariable> pair)
+            throws AlgebricksException {
+        op.getMaxObjects().getValue().substituteVar(pair.first, pair.second);
+        ILogicalExpression offset = op.getOffset().getValue();
+        if (offset != null) {
+            offset.substituteVar(pair.first, pair.second);
+        }
+        substVarTypes(op, pair);
+        return null;
+    }
+
+    @Override
+    public Void visitNestedTupleSourceOperator(NestedTupleSourceOperator op, Pair<LogicalVariable, LogicalVariable> pair)
+            throws AlgebricksException {
+        return null;
+    }
+
+    @Override
+    public Void visitOrderOperator(OrderOperator op, Pair<LogicalVariable, LogicalVariable> pair)
+            throws AlgebricksException {
+        for (Pair<IOrder, Mutable<ILogicalExpression>> oe : op.getOrderExpressions()) {
+            oe.second.getValue().substituteVar(pair.first, pair.second);
+        }
+        substVarTypes(op, pair);
+        return null;
+    }
+
+    @Override
+    public Void visitPartitioningSplitOperator(PartitioningSplitOperator op, Pair<LogicalVariable, LogicalVariable> pair)
+            throws AlgebricksException {
+        for (Mutable<ILogicalExpression> e : op.getExpressions()) {
+            e.getValue().substituteVar(pair.first, pair.second);
+        }
+        substVarTypes(op, pair);
+        return null;
+    }
+
+    @Override
+    public Void visitProjectOperator(ProjectOperator op, Pair<LogicalVariable, LogicalVariable> pair)
+            throws AlgebricksException {
+        List<LogicalVariable> usedVariables = op.getVariables();
+        int n = usedVariables.size();
+        for (int i = 0; i < n; i++) {
+            LogicalVariable v = usedVariables.get(i);
+            if (v.equals(pair.first)) {
+                usedVariables.set(i, pair.second);
+            }
+        }
+        substVarTypes(op, pair);
+        return null;
+    }
+
+    @Override
+    public Void visitRunningAggregateOperator(RunningAggregateOperator op, Pair<LogicalVariable, LogicalVariable> pair)
+            throws AlgebricksException {
+        List<LogicalVariable> variables = op.getVariables();
+        int n = variables.size();
+        for (int i = 0; i < n; i++) {
+            if (variables.get(i).equals(pair.first)) {
+                variables.set(i, pair.second);
+            } else {
+                op.getExpressions().get(i).getValue().substituteVar(pair.first, pair.second);
+            }
+        }
+        substVarTypes(op, pair);
+        return null;
+    }
+
+    @Override
+    public Void visitScriptOperator(ScriptOperator op, Pair<LogicalVariable, LogicalVariable> pair)
+            throws AlgebricksException {
+        substInArray(op.getInputVariables(), pair.first, pair.second);
+        substInArray(op.getOutputVariables(), pair.first, pair.second);
+        substVarTypes(op, pair);
+        return null;
+    }
+
+    @Override
+    public Void visitSelectOperator(SelectOperator op, Pair<LogicalVariable, LogicalVariable> pair) {
+        op.getCondition().getValue().substituteVar(pair.first, pair.second);
+        return null;
+    }
+
+    @Override
+    public Void visitSubplanOperator(SubplanOperator op, Pair<LogicalVariable, LogicalVariable> pair)
+            throws AlgebricksException {
+        for (ILogicalPlan p : op.getNestedPlans()) {
+            for (Mutable<ILogicalOperator> r : p.getRoots()) {
+                OperatorManipulationUtil.substituteVarRec((AbstractLogicalOperator) r.getValue(), pair.first,
+                        pair.second, goThroughNts, ctx);
+            }
+        }
+        return null;
+    }
+
+    @Override
+    public Void visitUnionOperator(UnionAllOperator op, Pair<LogicalVariable, LogicalVariable> pair)
+            throws AlgebricksException {
+        List<Triple<LogicalVariable, LogicalVariable, LogicalVariable>> varMap = op.getVariableMappings();
+        for (Triple<LogicalVariable, LogicalVariable, LogicalVariable> t : varMap) {
+            if (t.first.equals(pair.first)) {
+                t.first = pair.second;
+            }
+            if (t.second.equals(pair.first)) {
+                t.second = pair.second;
+            }
+            if (t.third.equals(pair.first)) {
+                t.third = pair.second;
+            }
+        }
+        substVarTypes(op, pair);
+        return null;
+    }
+
+    @Override
+    public Void visitUnnestMapOperator(UnnestMapOperator op, Pair<LogicalVariable, LogicalVariable> pair)
+            throws AlgebricksException {
+        List<LogicalVariable> variables = op.getVariables();
+        for (int i = 0; i < variables.size(); i++) {
+            if (variables.get(i) == pair.first) {
+                variables.set(i, pair.second);
+                return null;
+            }
+        }
+        op.getExpressionRef().getValue().substituteVar(pair.first, pair.second);
+        substVarTypes(op, pair);
+        return null;
+    }
+
+    @Override
+    public Void visitUnnestOperator(UnnestOperator op, Pair<LogicalVariable, LogicalVariable> pair)
+            throws AlgebricksException {
+        List<LogicalVariable> variables = op.getVariables();
+        for (int i = 0; i < variables.size(); i++) {
+            if (variables.get(i) == pair.first) {
+                variables.set(i, pair.second);
+                return null;
+            }
+        }
+        op.getExpressionRef().getValue().substituteVar(pair.first, pair.second);
+        substVarTypes(op, pair);
+        return null;
+    }
+
+    @Override
+    public Void visitWriteOperator(WriteOperator op, Pair<LogicalVariable, LogicalVariable> pair)
+            throws AlgebricksException {
+        for (Mutable<ILogicalExpression> e : op.getExpressions()) {
+            e.getValue().substituteVar(pair.first, pair.second);
+        }
+        substVarTypes(op, pair);
+        return null;
+    }
+
+    @Override
+    public Void visitDistributeResultOperator(DistributeResultOperator op, Pair<LogicalVariable, LogicalVariable> pair)
+            throws AlgebricksException {
+        for (Mutable<ILogicalExpression> e : op.getExpressions()) {
+            e.getValue().substituteVar(pair.first, pair.second);
+        }
+        substVarTypes(op, pair);
+        return null;
+    }
+
+    @Override
+    public Void visitWriteResultOperator(WriteResultOperator op, Pair<LogicalVariable, LogicalVariable> pair)
+            throws AlgebricksException {
+        op.getPayloadExpression().getValue().substituteVar(pair.first, pair.second);
+        for (Mutable<ILogicalExpression> e : op.getKeyExpressions()) {
+            e.getValue().substituteVar(pair.first, pair.second);
+        }
+        substVarTypes(op, pair);
+        return null;
+    }
+
+    private void subst(LogicalVariable v1, LogicalVariable v2,
+            List<Pair<LogicalVariable, Mutable<ILogicalExpression>>> varExprPairList) {
+        for (Pair<LogicalVariable, Mutable<ILogicalExpression>> ve : varExprPairList) {
+            if (ve.first != null && ve.first.equals(v1)) {
+                ve.first = v2;
+                return;
+            }
+            ve.second.getValue().substituteVar(v1, v2);
+        }
+    }
+
+    private void substInArray(ArrayList<LogicalVariable> varArray, LogicalVariable v1, LogicalVariable v2) {
+        for (int i = 0; i < varArray.size(); i++) {
+            LogicalVariable v = varArray.get(i);
+            if (v == v1) {
+                varArray.set(i, v2);
+            }
+        }
+    }
+
+    @Override
+    public Void visitReplicateOperator(ReplicateOperator op, Pair<LogicalVariable, LogicalVariable> arg)
+            throws AlgebricksException {
+        op.substituteVar(arg.first, arg.second);
+        return null;
+    }
+
+    @Override
+    public Void visitInsertDeleteOperator(InsertDeleteOperator op, Pair<LogicalVariable, LogicalVariable> pair)
+            throws AlgebricksException {
+        op.getPayloadExpression().getValue().substituteVar(pair.first, pair.second);
+        for (Mutable<ILogicalExpression> e : op.getPrimaryKeyExpressions()) {
+            e.getValue().substituteVar(pair.first, pair.second);
+        }
+        substVarTypes(op, pair);
+        return null;
+    }
+
+    @Override
+    public Void visitIndexInsertDeleteOperator(IndexInsertDeleteOperator op, Pair<LogicalVariable, LogicalVariable> pair)
+            throws AlgebricksException {
+        for (Mutable<ILogicalExpression> e : op.getPrimaryKeyExpressions()) {
+            e.getValue().substituteVar(pair.first, pair.second);
+        }
+        for (Mutable<ILogicalExpression> e : op.getSecondaryKeyExpressions()) {
+            e.getValue().substituteVar(pair.first, pair.second);
+        }
+        substVarTypes(op, pair);
+        return null;
+    }
+
+    @Override
+    public Void visitSinkOperator(SinkOperator op, Pair<LogicalVariable, LogicalVariable> pair)
+            throws AlgebricksException {
+        return null;
+    }
+
+    private void substVarTypes(ILogicalOperator op, Pair<LogicalVariable, LogicalVariable> arg)
+            throws AlgebricksException {
+        if (ctx == null) {
+            return;
+        }
+        IVariableTypeEnvironment env = ctx.getOutputTypeEnvironment(op);
+        env.substituteProducedVariable(arg.first, arg.second);
+    }
+
+    @Override
+    public Void visitExtensionOperator(ExtensionOperator op, Pair<LogicalVariable, LogicalVariable> arg)
+            throws AlgebricksException {
+        return null;
+    }
+
+    @Override
+    public Void visitExternalDataLookupOperator(ExternalDataLookupOperator op,
+            Pair<LogicalVariable, LogicalVariable> pair) throws AlgebricksException {
+        List<LogicalVariable> variables = op.getVariables();
+        for (int i = 0; i < variables.size(); i++) {
+            if (variables.get(i) == pair.first) {
+                variables.set(i, pair.second);
+                return null;
+            }
+        }
+        op.getExpressionRef().getValue().substituteVar(pair.first, pair.second);
+        substVarTypes(op, pair);
+        return null;
+    }
+}
diff --git a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/visitors/UsedVariableVisitor.java b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/visitors/UsedVariableVisitor.java
index d05adfb..8dc545f 100644
--- a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/visitors/UsedVariableVisitor.java
+++ b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/visitors/UsedVariableVisitor.java
@@ -12,339 +12,346 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.visitors;

-

-import java.util.Collection;

-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.common.utils.Pair;

-import edu.uci.ics.hyracks.algebricks.common.utils.Triple;

-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.IPhysicalOperator;

-import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;

-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AggregateOperator;

-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AssignOperator;

-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.DataSourceScanOperator;

-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.DistinctOperator;

-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.DistributeResultOperator;

-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.EmptyTupleSourceOperator;

-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ExchangeOperator;

-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ExtensionOperator;

-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.GroupByOperator;

-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.IndexInsertDeleteOperator;

-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.InnerJoinOperator;

-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.NestedTupleSourceOperator;

-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.OrderOperator;

-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.OrderOperator.IOrder;

-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.PartitioningSplitOperator;

-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ProjectOperator;

-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ReplicateOperator;

-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.RunningAggregateOperator;

-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ScriptOperator;

-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.SelectOperator;

-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.SinkOperator;

-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.SubplanOperator;

-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.UnionAllOperator;

-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.UnnestMapOperator;

-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.UnnestOperator;

-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.WriteOperator;

-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.WriteResultOperator;

-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical.HashPartitionExchangePOperator;

-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical.HashPartitionMergeExchangePOperator;

-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical.RangePartitionPOperator;

-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical.SortMergeExchangePOperator;

-import edu.uci.ics.hyracks.algebricks.core.algebra.properties.OrderColumn;

-import edu.uci.ics.hyracks.algebricks.core.algebra.visitors.ILogicalOperatorVisitor;

-

-public class UsedVariableVisitor implements ILogicalOperatorVisitor<Void, Void> {

-

-    private Collection<LogicalVariable> usedVariables;

-

-    public UsedVariableVisitor(Collection<LogicalVariable> usedVariables) {

-        this.usedVariables = usedVariables;

-    }

-

-    @Override

-    public Void visitAggregateOperator(AggregateOperator op, Void arg) {

-        for (Mutable<ILogicalExpression> exprRef : op.getExpressions()) {

-            exprRef.getValue().getUsedVariables(usedVariables);

-        }

-        return null;

-    }

-

-    @Override

-    public Void visitAssignOperator(AssignOperator op, Void arg) {

-        for (Mutable<ILogicalExpression> exprRef : op.getExpressions()) {

-            exprRef.getValue().getUsedVariables(usedVariables);

-        }

-        return null;

-    }

-

-    @Override

-    public Void visitDataScanOperator(DataSourceScanOperator op, Void arg) {

-        // does not use any variable

-        return null;

-    }

-

-    @Override

-    public Void visitDistinctOperator(DistinctOperator op, Void arg) {

-        for (Mutable<ILogicalExpression> eRef : op.getExpressions()) {

-            eRef.getValue().getUsedVariables(usedVariables);

-        }

-        return null;

-    }

-

-    @Override

-    public Void visitEmptyTupleSourceOperator(EmptyTupleSourceOperator op, Void arg) {

-        // does not use any variable

-        return null;

-    }

-

-    @Override

-    public Void visitExchangeOperator(ExchangeOperator op, Void arg) throws AlgebricksException {

-        // Used variables depend on the physical operator.

-        if (op.getPhysicalOperator() != null) {

-            IPhysicalOperator physOp = op.getPhysicalOperator();

-            switch (physOp.getOperatorTag()) {

-                case BROADCAST_EXCHANGE:

-                case ONE_TO_ONE_EXCHANGE:

-                case RANDOM_MERGE_EXCHANGE: {

-                    // No variables used.

-                    break;

-                }

-                case HASH_PARTITION_EXCHANGE: {

-                    HashPartitionExchangePOperator concreteOp = (HashPartitionExchangePOperator) physOp;

-                    usedVariables.addAll(concreteOp.getHashFields());

-                    break;

-                }

-                case HASH_PARTITION_MERGE_EXCHANGE: {

-                    HashPartitionMergeExchangePOperator concreteOp = (HashPartitionMergeExchangePOperator) physOp;

-                    usedVariables.addAll(concreteOp.getPartitionFields());

-                    for (OrderColumn orderCol : concreteOp.getOrderColumns()) {

-                        usedVariables.add(orderCol.getColumn());

-                    }

-                    break;

-                }

-                case SORT_MERGE_EXCHANGE: {

-                    SortMergeExchangePOperator concreteOp = (SortMergeExchangePOperator) physOp;

-                    for (OrderColumn orderCol : concreteOp.getSortColumns()) {

-                        usedVariables.add(orderCol.getColumn());

-                    }

-                    break;

-                }

-                case RANGE_PARTITION_EXCHANGE: {

-                    RangePartitionPOperator concreteOp = (RangePartitionPOperator) physOp;

-                    for (OrderColumn partCol : concreteOp.getPartitioningFields()) {

-                        usedVariables.add(partCol.getColumn());

-                    }

-                    break;

-                }

-                default: {

-                    throw new AlgebricksException("Unhandled physical operator tag '" + physOp.getOperatorTag() + "'.");

-                }

-            }

-        }

-        return null;

-    }

-

-    @Override

-    public Void visitGroupByOperator(GroupByOperator op, Void arg) throws AlgebricksException {

-        for (ILogicalPlan p : op.getNestedPlans()) {

-            for (Mutable<ILogicalOperator> r : p.getRoots()) {

-                VariableUtilities.getUsedVariablesInDescendantsAndSelf(r.getValue(), usedVariables);

-            }

-        }

-        for (Pair<LogicalVariable, Mutable<ILogicalExpression>> g : op.getGroupByList()) {

-            g.second.getValue().getUsedVariables(usedVariables);

-        }

-        for (Pair<LogicalVariable, Mutable<ILogicalExpression>> g : op.getDecorList()) {

-            g.second.getValue().getUsedVariables(usedVariables);

-        }

-        return null;

-    }

-

-    @Override

-    public Void visitInnerJoinOperator(InnerJoinOperator op, Void arg) {

-        op.getCondition().getValue().getUsedVariables(usedVariables);

-        return null;

-    }

-

-    @Override

-    public Void visitLeftOuterJoinOperator(LeftOuterJoinOperator op, Void arg) {

-        op.getCondition().getValue().getUsedVariables(usedVariables);

-        return null;

-    }

-

-    @Override

-    public Void visitLimitOperator(LimitOperator op, Void arg) {

-        op.getMaxObjects().getValue().getUsedVariables(usedVariables);

-        ILogicalExpression offsetExpr = op.getOffset().getValue();

-        if (offsetExpr != null) {

-            offsetExpr.getUsedVariables(usedVariables);

-        }

-        return null;

-    }

-

-    @Override

-    public Void visitNestedTupleSourceOperator(NestedTupleSourceOperator op, Void arg) {

-        // does not use any variable

-        return null;

-    }

-

-    @Override

-    public Void visitOrderOperator(OrderOperator op, Void arg) {

-        for (Pair<IOrder, Mutable<ILogicalExpression>> oe : op.getOrderExpressions()) {

-            oe.second.getValue().getUsedVariables(usedVariables);

-        }

-        return null;

-    }

-

-    @Override

-    public Void visitPartitioningSplitOperator(PartitioningSplitOperator op, Void arg) {

-        for (Mutable<ILogicalExpression> e : op.getExpressions()) {

-            e.getValue().getUsedVariables(usedVariables);

-        }

-        return null;

-    }

-

-    @Override

-    public Void visitProjectOperator(ProjectOperator op, Void arg) {

-        List<LogicalVariable> parameterVariables = op.getVariables();

-        for (LogicalVariable v : parameterVariables) {

-            if (!usedVariables.contains(v)) {

-                usedVariables.add(v);

-            }

-        }

-        return null;

-    }

-

-    @Override

-    public Void visitRunningAggregateOperator(RunningAggregateOperator op, Void arg) {

-        for (Mutable<ILogicalExpression> exprRef : op.getExpressions()) {

-            exprRef.getValue().getUsedVariables(usedVariables);

-        }

-        return null;

-    }

-

-    @Override

-    public Void visitScriptOperator(ScriptOperator op, Void arg) {

-        List<LogicalVariable> parameterVariables = op.getInputVariables();

-        for (LogicalVariable v : parameterVariables) {

-            if (!usedVariables.contains(v)) {

-                usedVariables.add(v);

-            }

-        }

-        return null;

-    }

-

-    @Override

-    public Void visitSelectOperator(SelectOperator op, Void arg) {

-        op.getCondition().getValue().getUsedVariables(usedVariables);

-        return null;

-    }

-

-    @Override

-    public Void visitSubplanOperator(SubplanOperator op, Void arg) throws AlgebricksException {

-        for (ILogicalPlan p : op.getNestedPlans()) {

-            for (Mutable<ILogicalOperator> r : p.getRoots()) {

-                VariableUtilities.getUsedVariablesInDescendantsAndSelf(r.getValue(), usedVariables);

-            }

-        }

-        return null;

-    }

-

-    @Override

-    public Void visitUnionOperator(UnionAllOperator op, Void arg) {

-        for (Triple<LogicalVariable, LogicalVariable, LogicalVariable> m : op.getVariableMappings()) {

-            if (!usedVariables.contains(m.first)) {

-                usedVariables.add(m.first);

-            }

-            if (!usedVariables.contains(m.second)) {

-                usedVariables.add(m.second);

-            }

-        }

-        return null;

-    }

-

-    @Override

-    public Void visitUnnestMapOperator(UnnestMapOperator op, Void arg) {

-        op.getExpressionRef().getValue().getUsedVariables(usedVariables);

-        return null;

-    }

-

-    @Override

-    public Void visitUnnestOperator(UnnestOperator op, Void arg) {

-        op.getExpressionRef().getValue().getUsedVariables(usedVariables);

-        return null;

-    }

-

-    @Override

-    public Void visitWriteOperator(WriteOperator op, Void arg) {

-        for (Mutable<ILogicalExpression> expr : op.getExpressions()) {

-            expr.getValue().getUsedVariables(usedVariables);

-        }

-        return null;

-    }

-

-    @Override

-    public Void visitDistributeResultOperator(DistributeResultOperator op, Void arg) {

-        for (Mutable<ILogicalExpression> expr : op.getExpressions()) {

-            expr.getValue().getUsedVariables(usedVariables);

-        }

-        return null;

-    }

-

-    @Override

-    public Void visitWriteResultOperator(WriteResultOperator op, Void arg) {

-        op.getPayloadExpression().getValue().getUsedVariables(usedVariables);

-        for (Mutable<ILogicalExpression> e : op.getKeyExpressions()) {

-            e.getValue().getUsedVariables(usedVariables);

-        }

-        return null;

-    }

-

-    @Override

-    public Void visitInsertDeleteOperator(InsertDeleteOperator op, Void arg) {

-        op.getPayloadExpression().getValue().getUsedVariables(usedVariables);

-        for (Mutable<ILogicalExpression> e : op.getPrimaryKeyExpressions()) {

-            e.getValue().getUsedVariables(usedVariables);

-        }

-        return null;

-    }

-

-    @Override

-    public Void visitIndexInsertDeleteOperator(IndexInsertDeleteOperator op, Void arg) {

-        for (Mutable<ILogicalExpression> e : op.getPrimaryKeyExpressions()) {

-            e.getValue().getUsedVariables(usedVariables);

-        }

-        for (Mutable<ILogicalExpression> e : op.getSecondaryKeyExpressions()) {

-            e.getValue().getUsedVariables(usedVariables);

-        }

-        return null;

-    }

-

-    @Override

-    public Void visitSinkOperator(SinkOperator op, Void arg) {

-        return null;

-    }

-

-    @Override

-    public Void visitReplicateOperator(ReplicateOperator op, Void arg) throws AlgebricksException {

-        return null;

-    }

-

-    @Override

-    public Void visitExtensionOperator(ExtensionOperator op, Void arg) throws AlgebricksException {

-        op.getDelegate().getUsedVariables(usedVariables);

-        return null;

-    }

-

-}

+package edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.visitors;
+
+import java.util.Collection;
+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.common.utils.Pair;
+import edu.uci.ics.hyracks.algebricks.common.utils.Triple;
+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.IPhysicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AggregateOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AssignOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.DataSourceScanOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.DistinctOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.DistributeResultOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.EmptyTupleSourceOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ExchangeOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ExtensionOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ExternalDataLookupOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.GroupByOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.IndexInsertDeleteOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.InnerJoinOperator;
+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.NestedTupleSourceOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.OrderOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.OrderOperator.IOrder;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.PartitioningSplitOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ProjectOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ReplicateOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.RunningAggregateOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ScriptOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.SelectOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.SinkOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.SubplanOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.UnionAllOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.UnnestMapOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.UnnestOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.WriteOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.WriteResultOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical.HashPartitionExchangePOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical.HashPartitionMergeExchangePOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical.RangePartitionPOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical.SortMergeExchangePOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.OrderColumn;
+import edu.uci.ics.hyracks.algebricks.core.algebra.visitors.ILogicalOperatorVisitor;
+
+public class UsedVariableVisitor implements ILogicalOperatorVisitor<Void, Void> {
+
+    private Collection<LogicalVariable> usedVariables;
+
+    public UsedVariableVisitor(Collection<LogicalVariable> usedVariables) {
+        this.usedVariables = usedVariables;
+    }
+
+    @Override
+    public Void visitAggregateOperator(AggregateOperator op, Void arg) {
+        for (Mutable<ILogicalExpression> exprRef : op.getExpressions()) {
+            exprRef.getValue().getUsedVariables(usedVariables);
+        }
+        return null;
+    }
+
+    @Override
+    public Void visitAssignOperator(AssignOperator op, Void arg) {
+        for (Mutable<ILogicalExpression> exprRef : op.getExpressions()) {
+            exprRef.getValue().getUsedVariables(usedVariables);
+        }
+        return null;
+    }
+
+    @Override
+    public Void visitDataScanOperator(DataSourceScanOperator op, Void arg) {
+        // does not use any variable
+        return null;
+    }
+
+    @Override
+    public Void visitDistinctOperator(DistinctOperator op, Void arg) {
+        for (Mutable<ILogicalExpression> eRef : op.getExpressions()) {
+            eRef.getValue().getUsedVariables(usedVariables);
+        }
+        return null;
+    }
+
+    @Override
+    public Void visitEmptyTupleSourceOperator(EmptyTupleSourceOperator op, Void arg) {
+        // does not use any variable
+        return null;
+    }
+
+    @Override
+    public Void visitExchangeOperator(ExchangeOperator op, Void arg) throws AlgebricksException {
+        // Used variables depend on the physical operator.
+        if (op.getPhysicalOperator() != null) {
+            IPhysicalOperator physOp = op.getPhysicalOperator();
+            switch (physOp.getOperatorTag()) {
+                case BROADCAST_EXCHANGE:
+                case ONE_TO_ONE_EXCHANGE:
+                case RANDOM_MERGE_EXCHANGE: {
+                    // No variables used.
+                    break;
+                }
+                case HASH_PARTITION_EXCHANGE: {
+                    HashPartitionExchangePOperator concreteOp = (HashPartitionExchangePOperator) physOp;
+                    usedVariables.addAll(concreteOp.getHashFields());
+                    break;
+                }
+                case HASH_PARTITION_MERGE_EXCHANGE: {
+                    HashPartitionMergeExchangePOperator concreteOp = (HashPartitionMergeExchangePOperator) physOp;
+                    usedVariables.addAll(concreteOp.getPartitionFields());
+                    for (OrderColumn orderCol : concreteOp.getOrderColumns()) {
+                        usedVariables.add(orderCol.getColumn());
+                    }
+                    break;
+                }
+                case SORT_MERGE_EXCHANGE: {
+                    SortMergeExchangePOperator concreteOp = (SortMergeExchangePOperator) physOp;
+                    for (OrderColumn orderCol : concreteOp.getSortColumns()) {
+                        usedVariables.add(orderCol.getColumn());
+                    }
+                    break;
+                }
+                case RANGE_PARTITION_EXCHANGE: {
+                    RangePartitionPOperator concreteOp = (RangePartitionPOperator) physOp;
+                    for (OrderColumn partCol : concreteOp.getPartitioningFields()) {
+                        usedVariables.add(partCol.getColumn());
+                    }
+                    break;
+                }
+                default: {
+                    throw new AlgebricksException("Unhandled physical operator tag '" + physOp.getOperatorTag() + "'.");
+                }
+            }
+        }
+        return null;
+    }
+
+    @Override
+    public Void visitGroupByOperator(GroupByOperator op, Void arg) throws AlgebricksException {
+        for (ILogicalPlan p : op.getNestedPlans()) {
+            for (Mutable<ILogicalOperator> r : p.getRoots()) {
+                VariableUtilities.getUsedVariablesInDescendantsAndSelf(r.getValue(), usedVariables);
+            }
+        }
+        for (Pair<LogicalVariable, Mutable<ILogicalExpression>> g : op.getGroupByList()) {
+            g.second.getValue().getUsedVariables(usedVariables);
+        }
+        for (Pair<LogicalVariable, Mutable<ILogicalExpression>> g : op.getDecorList()) {
+            g.second.getValue().getUsedVariables(usedVariables);
+        }
+        return null;
+    }
+
+    @Override
+    public Void visitInnerJoinOperator(InnerJoinOperator op, Void arg) {
+        op.getCondition().getValue().getUsedVariables(usedVariables);
+        return null;
+    }
+
+    @Override
+    public Void visitLeftOuterJoinOperator(LeftOuterJoinOperator op, Void arg) {
+        op.getCondition().getValue().getUsedVariables(usedVariables);
+        return null;
+    }
+
+    @Override
+    public Void visitLimitOperator(LimitOperator op, Void arg) {
+        op.getMaxObjects().getValue().getUsedVariables(usedVariables);
+        ILogicalExpression offsetExpr = op.getOffset().getValue();
+        if (offsetExpr != null) {
+            offsetExpr.getUsedVariables(usedVariables);
+        }
+        return null;
+    }
+
+    @Override
+    public Void visitNestedTupleSourceOperator(NestedTupleSourceOperator op, Void arg) {
+        // does not use any variable
+        return null;
+    }
+
+    @Override
+    public Void visitOrderOperator(OrderOperator op, Void arg) {
+        for (Pair<IOrder, Mutable<ILogicalExpression>> oe : op.getOrderExpressions()) {
+            oe.second.getValue().getUsedVariables(usedVariables);
+        }
+        return null;
+    }
+
+    @Override
+    public Void visitPartitioningSplitOperator(PartitioningSplitOperator op, Void arg) {
+        for (Mutable<ILogicalExpression> e : op.getExpressions()) {
+            e.getValue().getUsedVariables(usedVariables);
+        }
+        return null;
+    }
+
+    @Override
+    public Void visitProjectOperator(ProjectOperator op, Void arg) {
+        List<LogicalVariable> parameterVariables = op.getVariables();
+        for (LogicalVariable v : parameterVariables) {
+            if (!usedVariables.contains(v)) {
+                usedVariables.add(v);
+            }
+        }
+        return null;
+    }
+
+    @Override
+    public Void visitRunningAggregateOperator(RunningAggregateOperator op, Void arg) {
+        for (Mutable<ILogicalExpression> exprRef : op.getExpressions()) {
+            exprRef.getValue().getUsedVariables(usedVariables);
+        }
+        return null;
+    }
+
+    @Override
+    public Void visitScriptOperator(ScriptOperator op, Void arg) {
+        List<LogicalVariable> parameterVariables = op.getInputVariables();
+        for (LogicalVariable v : parameterVariables) {
+            if (!usedVariables.contains(v)) {
+                usedVariables.add(v);
+            }
+        }
+        return null;
+    }
+
+    @Override
+    public Void visitSelectOperator(SelectOperator op, Void arg) {
+        op.getCondition().getValue().getUsedVariables(usedVariables);
+        return null;
+    }
+
+    @Override
+    public Void visitSubplanOperator(SubplanOperator op, Void arg) throws AlgebricksException {
+        for (ILogicalPlan p : op.getNestedPlans()) {
+            for (Mutable<ILogicalOperator> r : p.getRoots()) {
+                VariableUtilities.getUsedVariablesInDescendantsAndSelf(r.getValue(), usedVariables);
+            }
+        }
+        return null;
+    }
+
+    @Override
+    public Void visitUnionOperator(UnionAllOperator op, Void arg) {
+        for (Triple<LogicalVariable, LogicalVariable, LogicalVariable> m : op.getVariableMappings()) {
+            if (!usedVariables.contains(m.first)) {
+                usedVariables.add(m.first);
+            }
+            if (!usedVariables.contains(m.second)) {
+                usedVariables.add(m.second);
+            }
+        }
+        return null;
+    }
+
+    @Override
+    public Void visitUnnestMapOperator(UnnestMapOperator op, Void arg) {
+        op.getExpressionRef().getValue().getUsedVariables(usedVariables);
+        return null;
+    }
+
+    @Override
+    public Void visitUnnestOperator(UnnestOperator op, Void arg) {
+        op.getExpressionRef().getValue().getUsedVariables(usedVariables);
+        return null;
+    }
+
+    @Override
+    public Void visitWriteOperator(WriteOperator op, Void arg) {
+        for (Mutable<ILogicalExpression> expr : op.getExpressions()) {
+            expr.getValue().getUsedVariables(usedVariables);
+        }
+        return null;
+    }
+
+    @Override
+    public Void visitDistributeResultOperator(DistributeResultOperator op, Void arg) {
+        for (Mutable<ILogicalExpression> expr : op.getExpressions()) {
+            expr.getValue().getUsedVariables(usedVariables);
+        }
+        return null;
+    }
+
+    @Override
+    public Void visitWriteResultOperator(WriteResultOperator op, Void arg) {
+        op.getPayloadExpression().getValue().getUsedVariables(usedVariables);
+        for (Mutable<ILogicalExpression> e : op.getKeyExpressions()) {
+            e.getValue().getUsedVariables(usedVariables);
+        }
+        return null;
+    }
+
+    @Override
+    public Void visitInsertDeleteOperator(InsertDeleteOperator op, Void arg) {
+        op.getPayloadExpression().getValue().getUsedVariables(usedVariables);
+        for (Mutable<ILogicalExpression> e : op.getPrimaryKeyExpressions()) {
+            e.getValue().getUsedVariables(usedVariables);
+        }
+        return null;
+    }
+
+    @Override
+    public Void visitIndexInsertDeleteOperator(IndexInsertDeleteOperator op, Void arg) {
+        for (Mutable<ILogicalExpression> e : op.getPrimaryKeyExpressions()) {
+            e.getValue().getUsedVariables(usedVariables);
+        }
+        for (Mutable<ILogicalExpression> e : op.getSecondaryKeyExpressions()) {
+            e.getValue().getUsedVariables(usedVariables);
+        }
+        return null;
+    }
+
+    @Override
+    public Void visitSinkOperator(SinkOperator op, Void arg) {
+        return null;
+    }
+
+    @Override
+    public Void visitReplicateOperator(ReplicateOperator op, Void arg) throws AlgebricksException {
+        return null;
+    }
+
+    @Override
+    public Void visitExtensionOperator(ExtensionOperator op, Void arg) throws AlgebricksException {
+        op.getDelegate().getUsedVariables(usedVariables);
+        return null;
+    }
+
+    @Override
+    public Void visitExternalDataLookupOperator(ExternalDataLookupOperator op, Void arg) throws AlgebricksException {
+        op.getExpressionRef().getValue().getUsedVariables(usedVariables);
+        return null;
+    }
+
+}
diff --git a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/prettyprint/LogicalOperatorPrettyPrintVisitor.java b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/prettyprint/LogicalOperatorPrettyPrintVisitor.java
index 5a2619d..1ca6d26 100644
--- a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/prettyprint/LogicalOperatorPrettyPrintVisitor.java
+++ b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/prettyprint/LogicalOperatorPrettyPrintVisitor.java
@@ -33,6 +33,7 @@
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.EmptyTupleSourceOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ExchangeOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ExtensionOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ExternalDataLookupOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.GroupByOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.IndexInsertDeleteOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.InnerJoinOperator;
@@ -396,4 +397,13 @@
         sb.append("]");
     }
 
+    @Override
+    public String visitExternalDataLookupOperator(ExternalDataLookupOperator op, Integer indent)
+            throws AlgebricksException {
+        StringBuilder buffer = new StringBuilder();
+        addIndent(buffer, indent).append(
+                "external-instant-lookup " + op.getVariables() + " <- " + op.getExpressionRef().getValue());
+        return buffer.toString();
+    }
+
 }
diff --git a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/visitors/ILogicalOperatorVisitor.java b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/visitors/ILogicalOperatorVisitor.java
index 0467d63..3fdee94 100644
--- a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/visitors/ILogicalOperatorVisitor.java
+++ b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/visitors/ILogicalOperatorVisitor.java
@@ -23,6 +23,7 @@
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.EmptyTupleSourceOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ExchangeOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ExtensionOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ExternalDataLookupOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.GroupByOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.IndexInsertDeleteOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.InnerJoinOperator;
@@ -104,5 +105,7 @@
     public R visitInsertDeleteOperator(InsertDeleteOperator op, T tag) throws AlgebricksException;
 
     public R visitIndexInsertDeleteOperator(IndexInsertDeleteOperator op, T tag) throws AlgebricksException;
+    
+    public R visitExternalDataLookupOperator(ExternalDataLookupOperator op, T arg) throws AlgebricksException;
 
 }
diff --git a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/IIndex.java b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/IIndex.java
index 4006cb7..5ae96f0 100644
--- a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/IIndex.java
+++ b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/IIndex.java
@@ -122,4 +122,10 @@
      */
     public IIndexBulkLoader createBulkLoader(float fillFactor, boolean verifyInput, long numElementsHint,
             boolean checkIfEmptyIndex) throws IndexException;
+    
+    /**
+     * 
+     * @return true if the index needs memory components
+     */
+    public boolean hasMemoryComponents();
 }
diff --git a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/ITwoPCIndexBulkLoader.java b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/ITwoPCIndexBulkLoader.java
new file mode 100644
index 0000000..8fda095
--- /dev/null
+++ b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/ITwoPCIndexBulkLoader.java
@@ -0,0 +1,39 @@
+package edu.uci.ics.hyracks.storage.am.common.api;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
+
+/*
+ * 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.
+ */
+public interface ITwoPCIndexBulkLoader {
+    
+    /**
+     * Append a "delete" tuple to the index in the context of a bulk load.
+     * 
+     * @param tuple
+     *            "delete" Tuple to be inserted.
+     * @throws IndexException
+     *             If the input stream is invalid for bulk loading (e.g., is not sorted).
+     * @throws HyracksDataException
+     *             If the BufferCache throws while un/pinning or un/latching.
+     */
+    public void delete(ITupleReference tuple) throws IndexException, HyracksDataException;
+    
+    /**
+     * Abort the bulk modify op
+     */
+    public void abort();
+
+}
diff --git a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/IndexBulkLoadOperatorNodePushable.java b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/IndexBulkLoadOperatorNodePushable.java
index a0f9d8e..81c13a1 100644
--- a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/IndexBulkLoadOperatorNodePushable.java
+++ b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/IndexBulkLoadOperatorNodePushable.java
@@ -29,18 +29,18 @@
 import edu.uci.ics.hyracks.storage.am.common.tuples.PermutingFrameTupleReference;
 
 public class IndexBulkLoadOperatorNodePushable extends AbstractUnaryInputSinkOperatorNodePushable {
-    private final IIndexOperatorDescriptor opDesc;
-    private final IHyracksTaskContext ctx;
-    private final float fillFactor;
-    private final boolean verifyInput;
-    private final long numElementsHint;
-    private final boolean checkIfEmptyIndex;
-    private final IIndexDataflowHelper indexHelper;
-    private FrameTupleAccessor accessor;
-    private IIndex index;
-    private IIndexBulkLoader bulkLoader;
-    private IRecordDescriptorProvider recDescProvider;
-    private PermutingFrameTupleReference tuple = new PermutingFrameTupleReference();
+    protected final IIndexOperatorDescriptor opDesc;
+    protected final IHyracksTaskContext ctx;
+    protected final float fillFactor;
+    protected final boolean verifyInput;
+    protected final long numElementsHint;
+    protected final boolean checkIfEmptyIndex;
+    protected final IIndexDataflowHelper indexHelper;
+    protected FrameTupleAccessor accessor;
+    protected IIndex index;
+    protected IIndexBulkLoader bulkLoader;
+    protected IRecordDescriptorProvider recDescProvider;
+    protected PermutingFrameTupleReference tuple = new PermutingFrameTupleReference();
 
     public IndexBulkLoadOperatorNodePushable(IIndexOperatorDescriptor opDesc, IHyracksTaskContext ctx, int partition,
             int[] fieldPermutation, float fillFactor, boolean verifyInput, long numElementsHint,
diff --git a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/IndexSearchOperatorNodePushable.java b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/IndexSearchOperatorNodePushable.java
index acc5bc2..f2ff6eb 100644
--- a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/IndexSearchOperatorNodePushable.java
+++ b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/IndexSearchOperatorNodePushable.java
@@ -167,7 +167,6 @@
                 }
             }
         }
-
     }
 
     @Override
diff --git a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/impls/AbstractTreeIndex.java b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/impls/AbstractTreeIndex.java
index 517660b..7fdcd96 100644
--- a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/impls/AbstractTreeIndex.java
+++ b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/impls/AbstractTreeIndex.java
@@ -337,6 +337,14 @@
             interiorFrame.initBuffer((byte) nodeFrontiers.size());
             nodeFrontiers.add(frontier);
         }
+
+        public ITreeIndexFrame getLeafFrame() {
+            return leafFrame;
+        }
+
+        public void setLeafFrame(ITreeIndexFrame leafFrame) {
+            this.leafFrame = leafFrame;
+        }
     }
 
     public class TreeIndexInsertBulkLoader implements IIndexBulkLoader {
@@ -367,4 +375,13 @@
     public long getMemoryAllocationSize() {
         return 0;
     }
+
+    public IBinaryComparatorFactory[] getCmpFactories() {
+        return cmpFactories;
+    }
+    
+    @Override
+    public boolean hasMemoryComponents() {
+        return true;
+    }
 }
diff --git a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/tuples/TypeAwareTupleWriterFactory.java b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/tuples/TypeAwareTupleWriterFactory.java
index 9a916d6..896abb5 100644
--- a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/tuples/TypeAwareTupleWriterFactory.java
+++ b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/tuples/TypeAwareTupleWriterFactory.java
@@ -23,7 +23,7 @@
 		ITreeIndexTupleWriterFactory {
 
 	private static final long serialVersionUID = 1L;
-	private ITypeTraits[] typeTraits;
+	protected ITypeTraits[] typeTraits;
 
 	public TypeAwareTupleWriterFactory(ITypeTraits[] typeTraits) {
 		this.typeTraits = typeTraits;
diff --git a/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/dataflow/ExternalBTreeDataflowHelper.java b/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/dataflow/ExternalBTreeDataflowHelper.java
new file mode 100644
index 0000000..2a458ac
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/dataflow/ExternalBTreeDataflowHelper.java
@@ -0,0 +1,86 @@
+/*
+ * 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.hyracks.storage.am.lsm.btree.dataflow;
+
+import java.util.List;
+
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndex;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.AbstractTreeIndexOperatorDescriptor;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexOperatorDescriptor;
+import edu.uci.ics.hyracks.storage.am.lsm.btree.util.LSMBTreeUtils;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMMergePolicy;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerProvider;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.IVirtualBufferCache;
+
+public class ExternalBTreeDataflowHelper extends LSMBTreeDataflowHelper {
+
+    private int version;
+
+    public ExternalBTreeDataflowHelper(IIndexOperatorDescriptor opDesc, IHyracksTaskContext ctx, int partition,
+            double bloomFilterFalsePositiveRate, ILSMMergePolicy mergePolicy,
+            ILSMOperationTrackerProvider opTrackerFactory, ILSMIOOperationScheduler ioScheduler,
+            ILSMIOOperationCallbackFactory ioOpCallbackFactory, boolean needKeyDupCheck, int version) {
+        super(opDesc, ctx, partition, null, bloomFilterFalsePositiveRate, mergePolicy, opTrackerFactory, ioScheduler,
+                ioOpCallbackFactory, false);
+        this.version = version;
+    }
+
+    public ExternalBTreeDataflowHelper(IIndexOperatorDescriptor opDesc, IHyracksTaskContext ctx, int partition,
+            List<IVirtualBufferCache> virtualBufferCaches, ILSMMergePolicy mergePolicy,
+            ILSMOperationTrackerProvider opTrackerFactory, ILSMIOOperationScheduler ioScheduler,
+            ILSMIOOperationCallbackFactory ioOpCallbackFactory, boolean needKeyDupCheck, int version) {
+        this(opDesc, ctx, partition, DEFAULT_BLOOM_FILTER_FALSE_POSITIVE_RATE, mergePolicy, opTrackerFactory,
+                ioScheduler, ioOpCallbackFactory, needKeyDupCheck, version);
+    }
+
+    @Override
+    public IIndex getIndexInstance() {
+        if (index != null)
+            return index;
+        synchronized (lcManager) {
+            long resourceID;
+            try {
+                resourceID = getResourceID();
+            } catch (HyracksDataException e) {
+                return null;
+            }
+            try {
+                index = lcManager.getIndex(resourceID);
+            } catch (HyracksDataException e) {
+                return null;
+            }
+        }
+        return index;
+    }
+
+    @Override
+    public ITreeIndex createIndexInstance() throws HyracksDataException {
+        AbstractTreeIndexOperatorDescriptor treeOpDesc = (AbstractTreeIndexOperatorDescriptor) opDesc;
+        return LSMBTreeUtils.createExternalBTree(file, opDesc.getStorageManager().getBufferCache(ctx), opDesc
+                .getStorageManager().getFileMapProvider(ctx), treeOpDesc.getTreeIndexTypeTraits(), treeOpDesc
+                .getTreeIndexComparatorFactories(), treeOpDesc.getTreeIndexBloomFilterKeyFields(),
+                bloomFilterFalsePositiveRate, mergePolicy, opTrackerFactory.getOperationTracker(ctx), ioScheduler,
+                ioOpCallbackFactory.createIOOperationCallback(), getVersion());
+    }
+
+    public int getVersion() {
+        return version;
+    }
+}
diff --git a/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/dataflow/ExternalBTreeDataflowHelperFactory.java b/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/dataflow/ExternalBTreeDataflowHelperFactory.java
new file mode 100644
index 0000000..48dd41e
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/dataflow/ExternalBTreeDataflowHelperFactory.java
@@ -0,0 +1,51 @@
+/*
+ * 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.hyracks.storage.am.lsm.btree.dataflow;
+
+import java.util.Map;
+
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexDataflowHelper;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexOperatorDescriptor;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationSchedulerProvider;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMMergePolicyFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerProvider;
+import edu.uci.ics.hyracks.storage.am.lsm.common.dataflow.AbstractLSMIndexDataflowHelperFactory;
+
+public class ExternalBTreeDataflowHelperFactory extends AbstractLSMIndexDataflowHelperFactory {
+
+    private static final long serialVersionUID = 1L;
+
+    private int version;
+
+    public ExternalBTreeDataflowHelperFactory(ILSMMergePolicyFactory mergePolicyFactory,
+            Map<String, String> mergePolicyProperties, ILSMOperationTrackerProvider opTrackerFactory,
+            ILSMIOOperationSchedulerProvider ioSchedulerProvider, ILSMIOOperationCallbackFactory ioOpCallbackFactory,
+            double bloomFilterFalsePositiveRate, int version) {
+        super(null, mergePolicyFactory, mergePolicyProperties, opTrackerFactory, ioSchedulerProvider,
+                ioOpCallbackFactory, bloomFilterFalsePositiveRate);
+        this.version = version;
+    }
+
+    @Override
+    public IIndexDataflowHelper createIndexDataflowHelper(IIndexOperatorDescriptor opDesc, IHyracksTaskContext ctx,
+            int partition) {
+        return new ExternalBTreeDataflowHelper(opDesc, ctx, partition, bloomFilterFalsePositiveRate,
+                mergePolicyFactory.createMergePolicy(mergePolicyProperties), opTrackerFactory,
+                ioSchedulerProvider.getIOScheduler(ctx), ioOpCallbackFactory, false, version);
+    }
+
+}
diff --git a/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/dataflow/ExternalBTreeWithBuddyDataflowHelper.java b/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/dataflow/ExternalBTreeWithBuddyDataflowHelper.java
new file mode 100644
index 0000000..7fb5009
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/dataflow/ExternalBTreeWithBuddyDataflowHelper.java
@@ -0,0 +1,89 @@
+/*
+ * 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.hyracks.storage.am.lsm.btree.dataflow;
+
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndex;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.AbstractTreeIndexOperatorDescriptor;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexOperatorDescriptor;
+import edu.uci.ics.hyracks.storage.am.lsm.btree.util.LSMBTreeUtils;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMMergePolicy;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerProvider;
+import edu.uci.ics.hyracks.storage.am.lsm.common.dataflow.AbstractLSMIndexDataflowHelper;
+
+public class ExternalBTreeWithBuddyDataflowHelper extends AbstractLSMIndexDataflowHelper {
+
+    private int[] buddyBtreeFields;
+    private int version;
+
+    public ExternalBTreeWithBuddyDataflowHelper(IIndexOperatorDescriptor opDesc, IHyracksTaskContext ctx,
+            int partition, ILSMMergePolicy mergePolicy,
+            ILSMOperationTrackerProvider opTrackerFactory, ILSMIOOperationScheduler ioScheduler,
+            ILSMIOOperationCallbackFactory ioOpCallbackFactory, int[] buddyBtreeFields, int version) {
+        super(opDesc, ctx, partition, null, mergePolicy, opTrackerFactory, ioScheduler,
+                ioOpCallbackFactory);
+        this.buddyBtreeFields = buddyBtreeFields;
+        this.version = version;
+    }
+
+    public ExternalBTreeWithBuddyDataflowHelper(IIndexOperatorDescriptor opDesc, IHyracksTaskContext ctx,
+            int partition, double bloomFilterFalsePositiveRate,
+            ILSMMergePolicy mergePolicy, ILSMOperationTrackerProvider opTrackerFactory,
+            ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallbackFactory ioOpCallbackFactory,
+            int[] buddyBtreeFields, int version) {
+        super(opDesc, ctx, partition, null, bloomFilterFalsePositiveRate, mergePolicy, opTrackerFactory,
+                ioScheduler, ioOpCallbackFactory);
+        this.buddyBtreeFields = buddyBtreeFields;
+        this.version = version;
+    }
+    
+    @Override
+    public IIndex getIndexInstance() {
+        if(index!= null)
+            return index;
+        synchronized (lcManager) {
+            long resourceID;
+            try {
+                resourceID = getResourceID();
+            } catch (HyracksDataException e) {
+                return null;
+            }
+            try {
+                index = lcManager.getIndex(resourceID);
+            } catch (HyracksDataException e) {
+                return null;
+            }
+        }
+        return index;
+    }
+
+    @Override
+    protected IIndex createIndexInstance() throws HyracksDataException {
+        AbstractTreeIndexOperatorDescriptor treeOpDesc = (AbstractTreeIndexOperatorDescriptor) opDesc;
+        return LSMBTreeUtils.createExternalBTreeWithBuddy(file, opDesc.getStorageManager().getBufferCache(ctx),
+                opDesc.getStorageManager().getFileMapProvider(ctx), treeOpDesc.getTreeIndexTypeTraits(),
+                treeOpDesc.getTreeIndexComparatorFactories(), bloomFilterFalsePositiveRate, mergePolicy,
+                opTrackerFactory.getOperationTracker(ctx), ioScheduler,
+                ioOpCallbackFactory.createIOOperationCallback(), buddyBtreeFields, version);
+    }
+    
+    public int getTargetVersion(){
+        return version;
+    }
+
+}
diff --git a/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/dataflow/ExternalBTreeWithBuddyDataflowHelperFactory.java b/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/dataflow/ExternalBTreeWithBuddyDataflowHelperFactory.java
new file mode 100644
index 0000000..67f0c85
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/dataflow/ExternalBTreeWithBuddyDataflowHelperFactory.java
@@ -0,0 +1,53 @@
+/*
+ * 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.hyracks.storage.am.lsm.btree.dataflow;
+
+import java.util.Map;
+
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexDataflowHelper;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexOperatorDescriptor;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationSchedulerProvider;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMMergePolicyFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerProvider;
+import edu.uci.ics.hyracks.storage.am.lsm.common.dataflow.AbstractLSMIndexDataflowHelperFactory;
+
+public class ExternalBTreeWithBuddyDataflowHelperFactory extends AbstractLSMIndexDataflowHelperFactory {
+
+    private static final long serialVersionUID = 1L;
+    private final int[] buddyBtreeFields;
+    private int version;
+
+    public ExternalBTreeWithBuddyDataflowHelperFactory(
+            ILSMMergePolicyFactory mergePolicyFactory, Map<String, String> mergePolicyProperties,
+            ILSMOperationTrackerProvider opTrackerFactory, ILSMIOOperationSchedulerProvider ioSchedulerProvider,
+            ILSMIOOperationCallbackFactory ioOpCallbackFactory, double bloomFilterFalsePositiveRate,
+            int[] buddyBtreeFields, int version) {
+        super(null, mergePolicyFactory, mergePolicyProperties, opTrackerFactory,
+                ioSchedulerProvider, ioOpCallbackFactory, bloomFilterFalsePositiveRate);
+        this.buddyBtreeFields = buddyBtreeFields;
+        this.version = version;
+    }
+
+    @Override
+    public IIndexDataflowHelper createIndexDataflowHelper(IIndexOperatorDescriptor opDesc, IHyracksTaskContext ctx,
+            int partition) {
+        return new ExternalBTreeWithBuddyDataflowHelper(opDesc, ctx, partition, bloomFilterFalsePositiveRate,
+                mergePolicyFactory.createMergePolicy(mergePolicyProperties), opTrackerFactory,
+                ioSchedulerProvider.getIOScheduler(ctx), ioOpCallbackFactory, buddyBtreeFields, version);
+    }
+
+}
diff --git a/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/impls/ExternalBTree.java b/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/impls/ExternalBTree.java
new file mode 100644
index 0000000..da19882
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/impls/ExternalBTree.java
@@ -0,0 +1,700 @@
+/*
+ * 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.hyracks.storage.am.lsm.btree.impls;
+
+import java.io.IOException;
+import java.util.LinkedList;
+import java.util.List;
+
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.io.FileReference;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
+import edu.uci.ics.hyracks.storage.am.bloomfilter.impls.BloomCalculations;
+import edu.uci.ics.hyracks.storage.am.bloomfilter.impls.BloomFilter;
+import edu.uci.ics.hyracks.storage.am.bloomfilter.impls.BloomFilterFactory;
+import edu.uci.ics.hyracks.storage.am.bloomfilter.impls.BloomFilterSpecification;
+import edu.uci.ics.hyracks.storage.am.btree.impls.BTree;
+import edu.uci.ics.hyracks.storage.am.btree.impls.BTree.BTreeBulkLoader;
+import edu.uci.ics.hyracks.storage.am.common.api.IFreePageManager;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexBulkLoader;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexCursor;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexOperationContext;
+import edu.uci.ics.hyracks.storage.am.common.api.IModificationOperationCallback;
+import edu.uci.ics.hyracks.storage.am.common.api.ISearchOperationCallback;
+import edu.uci.ics.hyracks.storage.am.common.api.ISearchPredicate;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexCursor;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexTupleWriterFactory;
+import edu.uci.ics.hyracks.storage.am.common.api.ITwoPCIndexBulkLoader;
+import edu.uci.ics.hyracks.storage.am.common.api.IndexException;
+import edu.uci.ics.hyracks.storage.am.common.api.TreeIndexException;
+import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallback;
+import edu.uci.ics.hyracks.storage.am.common.ophelpers.IndexOperation;
+import edu.uci.ics.hyracks.storage.am.lsm.btree.tuples.LSMBTreeRefrencingTupleWriterFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ITwoPCIndex;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMComponent;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndexAccessorInternal;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndexFileManager;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndexOperationContext;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMMergePolicy;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTracker;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.BlockingIOOperationCallbackWrapper;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.LSMOperationType;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.ExternalIndexHarness;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.LSMComponentFileReferences;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.TreeIndexFactory;
+import edu.uci.ics.hyracks.storage.common.file.IFileMapProvider;
+
+/**
+ * This is an lsm b-tree that does not have memory component and is modified
+ * only by bulk loading and addition of disk components as of this point, it is
+ * intended for use with external dataset indexes only.
+ * 
+ * @author alamouda
+ */
+public class ExternalBTree extends LSMBTree implements ITwoPCIndex {
+
+    // This component factory has to be different since it uses different tuple
+    // writer in it's leaf frames to support inserting both
+    // regular and delete tuples
+    private final LSMBTreeDiskComponentFactory transactionComponentFactory;
+    // A second disk component list that will be used when a transaction is
+    // committed and will be seen by subsequent accessors
+    private List<ILSMComponent> secondDiskComponents;
+    // A pointer that points to the current most recent list (either
+    // diskComponents = 0, or secondDiskComponents = 1). It starts with -1 to
+    // indicate first time activation
+    private int version = -1;
+
+    private final ITreeIndexFrameFactory interiorFrameFactory;
+
+    public ExternalBTree(ITreeIndexFrameFactory interiorFrameFactory, ITreeIndexFrameFactory insertLeafFrameFactory,
+            ITreeIndexFrameFactory deleteLeafFrameFactory, ILSMIndexFileManager fileManager,
+            TreeIndexFactory<BTree> diskBTreeFactory, TreeIndexFactory<BTree> bulkLoadBTreeFactory,
+            BloomFilterFactory bloomFilterFactory, double bloomFilterFalsePositiveRate,
+            IFileMapProvider diskFileMapProvider, int fieldCount, IBinaryComparatorFactory[] cmpFactories,
+            ILSMMergePolicy mergePolicy, ILSMOperationTracker opTracker, ILSMIOOperationScheduler ioScheduler,
+            ILSMIOOperationCallback ioOpCallback, TreeIndexFactory<BTree> transactionBTreeFactory, int version) {
+        super(interiorFrameFactory, insertLeafFrameFactory, deleteLeafFrameFactory, fileManager, diskBTreeFactory,
+                bulkLoadBTreeFactory, bloomFilterFactory, bloomFilterFalsePositiveRate, diskFileMapProvider,
+                fieldCount, cmpFactories, mergePolicy, opTracker, ioScheduler, ioOpCallback, false);
+        this.transactionComponentFactory = new LSMBTreeDiskComponentFactory(transactionBTreeFactory, bloomFilterFactory);
+        this.secondDiskComponents = new LinkedList<ILSMComponent>();
+        this.interiorFrameFactory = interiorFrameFactory;
+        this.version = version;
+    }
+
+    // This method is used to create a target for a bulk modify operation. This
+    // component must then be either committed or deleted
+    private ILSMComponent createTransactionTarget() throws HyracksDataException, IndexException {
+        LSMComponentFileReferences componentFileRefs;
+        try {
+            componentFileRefs = fileManager.getNewTransactionFileReference();
+        } catch (IOException e) {
+            throw new HyracksDataException("Failed to create transaction components", e);
+        }
+        return createDiskComponent(transactionComponentFactory, componentFileRefs.getInsertIndexFileReference(),
+                componentFileRefs.getBloomFilterFileReference(), true);
+    }
+
+    // The subsume merged components is overridden to account for:
+    // Maintaining two versions of the index
+    @Override
+    public void subsumeMergedComponents(ILSMComponent newComponent, List<ILSMComponent> mergedComponents)
+            throws HyracksDataException {
+        List<ILSMComponent> newerList;
+        List<ILSMComponent> olderList;
+        if (version == 0) {
+            newerList = diskComponents;
+            olderList = secondDiskComponents;
+        } else {
+            newerList = secondDiskComponents;
+            olderList = diskComponents;
+        }
+
+        // Check if merge will affect the older list
+        if (olderList.containsAll(mergedComponents)) {
+            int swapIndex = olderList.indexOf(mergedComponents.get(0));
+            olderList.removeAll(mergedComponents);
+            olderList.add(swapIndex, newComponent);
+        }
+
+        // The new list will always have all the merged components
+        int swapIndex = newerList.indexOf(mergedComponents.get(0));
+        newerList.removeAll(mergedComponents);
+        newerList.add(swapIndex, newComponent);
+    }
+
+    // This method is used by the merge policy when it needs to check if a merge
+    // is needed.
+    // It only needs to return the newer list
+    @Override
+    public List<ILSMComponent> getImmutableComponents() {
+        if (version == 0) {
+            return diskComponents;
+        } else {
+            return secondDiskComponents;
+        }
+    }
+
+    // The only reason to override the following method is that it uses a different context object
+    @Override
+    public void search(ILSMIndexOperationContext ictx, IIndexCursor cursor, ISearchPredicate pred)
+            throws HyracksDataException, IndexException {
+        ExternalBTreeOpContext ctx = (ExternalBTreeOpContext) ictx;
+        List<ILSMComponent> operationalComponents = ctx.getComponentHolder();
+        LSMBTreeCursorInitialState initialState = new LSMBTreeCursorInitialState(insertLeafFrameFactory, ctx.cmp,
+                ctx.bloomFilterCmp, lsmHarness, pred, ctx.searchCallback, operationalComponents);
+        cursor.open(initialState, pred);
+    }
+
+    // The only reason to override the following method is that it uses a different context object
+    // in addition, determining whether or not to keep deleted tuples is different here
+    @Override
+    public void scheduleMerge(ILSMIndexOperationContext ctx, ILSMIOOperationCallback callback)
+            throws HyracksDataException, IndexException {
+        ExternalBTreeOpContext opCtx = createOpContext(NoOpOperationCallback.INSTANCE, -1);
+        opCtx.setOperation(IndexOperation.MERGE);
+        List<ILSMComponent> mergingComponents = ctx.getComponentHolder();
+        boolean returnDeletedTuples = false;
+        if (version == 0) {
+            if (ctx.getComponentHolder().get(ctx.getComponentHolder().size() - 1) != diskComponents.get(diskComponents
+                    .size() - 1)) {
+                returnDeletedTuples = true;
+            }
+        } else {
+            if (ctx.getComponentHolder().get(ctx.getComponentHolder().size() - 1) != secondDiskComponents
+                    .get(secondDiskComponents.size() - 1)) {
+                returnDeletedTuples = true;
+            }
+        }
+        ITreeIndexCursor cursor = new LSMBTreeRangeSearchCursor(opCtx, returnDeletedTuples);
+        BTree firstBTree = (BTree) ((LSMBTreeDiskComponent) mergingComponents.get(0)).getBTree();
+        BTree lastBTree = (BTree) ((LSMBTreeDiskComponent) mergingComponents.get(mergingComponents.size() - 1))
+                .getBTree();
+        FileReference firstFile = diskFileMapProvider.lookupFileName(firstBTree.getFileId());
+        FileReference lastFile = diskFileMapProvider.lookupFileName(lastBTree.getFileId());
+        LSMComponentFileReferences relMergeFileRefs = fileManager.getRelMergeFileReference(firstFile.getFile()
+                .getName(), lastFile.getFile().getName());
+        ILSMIndexAccessorInternal accessor = new LSMBTreeAccessor(lsmHarness, opCtx);
+        ioScheduler.scheduleOperation(new LSMBTreeMergeOperation(accessor, mergingComponents, cursor, relMergeFileRefs
+                .getInsertIndexFileReference(), relMergeFileRefs.getBloomFilterFileReference(), callback, fileManager
+                .getBaseDir()));
+    }
+
+    // This function should only be used when a transaction fail. it doesn't
+    // take any parameters since there can only be
+    // a single transaction and hence a single transaction component on disk
+    public void deleteTransactionComponent() throws HyracksDataException {
+        fileManager.deleteTransactionFiles();
+    }
+
+    // This function in an instance of this index is only used after a bulk load
+    // is successful
+    @Override
+    public void addComponent(ILSMComponent c) throws HyracksDataException {
+        if (version == 0) {
+            diskComponents.add(0, c);
+        } else if (version == 1) {
+            secondDiskComponents.add(0, c);
+        }
+    }
+
+    // This function is used when a new component is to be committed.
+    public void commitTransactionDiskComponent(ILSMComponent newComponent) throws HyracksDataException {
+        // determine which list is the new one and flip the pointer
+        List<ILSMComponent> newerList;
+        List<ILSMComponent> olderList;
+        if (version == 0) {
+            newerList = diskComponents;
+            olderList = secondDiskComponents;
+            version = 1;
+        } else {
+            newerList = secondDiskComponents;
+            olderList = diskComponents;
+            version = 0;
+        }
+        // Remove components from list
+        olderList.clear();
+        // Add components
+        olderList.addAll(newerList);
+        if (newComponent != null) {
+            // Add it to the list
+            olderList.add(0, newComponent);
+        }
+    }
+
+    @Override
+    public synchronized void activate() throws HyracksDataException {
+        if (isActivated) {
+            throw new HyracksDataException("Failed to activate the index since it is already activated.");
+        }
+        if (diskComponents.size() == 0 && secondDiskComponents.size() == 0) {
+            //First time activation
+            List<LSMComponentFileReferences> validFileReferences;
+            try {
+                validFileReferences = fileManager.cleanupAndGetValidFiles();
+            } catch (IndexException e) {
+                throw new HyracksDataException(e);
+            }
+            for (LSMComponentFileReferences lsmComonentFileReference : validFileReferences) {
+                LSMBTreeDiskComponent component;
+                try {
+                    component = createDiskComponent(componentFactory,
+                            lsmComonentFileReference.getInsertIndexFileReference(),
+                            lsmComonentFileReference.getBloomFilterFileReference(), false);
+                } catch (IndexException e) {
+                    throw new HyracksDataException(e);
+                }
+                diskComponents.add(component);
+                secondDiskComponents.add(component);
+            }
+            ((ExternalIndexHarness) lsmHarness).indexFirstTimeActivated();
+        } else {
+            // This index has been opened before
+            for (ILSMComponent c : diskComponents) {
+                LSMBTreeDiskComponent component = (LSMBTreeDiskComponent) c;
+                BTree btree = component.getBTree();
+                BloomFilter bloomFilter = component.getBloomFilter();
+                btree.activate();
+                bloomFilter.activate();
+            }
+            for (ILSMComponent c : secondDiskComponents) {
+                // Only activate non shared components
+                if (!diskComponents.contains(c)) {
+                    LSMBTreeDiskComponent component = (LSMBTreeDiskComponent) c;
+                    BTree btree = component.getBTree();
+                    BloomFilter bloomFilter = component.getBloomFilter();
+                    btree.activate();
+                    bloomFilter.activate();
+                }
+            }
+        }
+        isActivated = true;
+    }
+
+    @Override
+    public synchronized void create() throws HyracksDataException {
+        super.create();
+        secondDiskComponents.clear();
+    }
+
+    @Override
+    public void deactivate(boolean flushOnExit) throws HyracksDataException {
+        if (!isActivated) {
+            throw new HyracksDataException("Failed to deactivate the index since it is already deactivated.");
+        }
+        if (flushOnExit) {
+            BlockingIOOperationCallbackWrapper cb = new BlockingIOOperationCallbackWrapper(ioOpCallback);
+            cb.afterFinalize(LSMOperationType.FLUSH, null);
+        }
+        for (ILSMComponent c : diskComponents) {
+            LSMBTreeDiskComponent component = (LSMBTreeDiskComponent) c;
+            BTree btree = component.getBTree();
+            BloomFilter bloomFilter = component.getBloomFilter();
+            btree.deactivate();
+            bloomFilter.deactivate();
+        }
+        for (ILSMComponent c : secondDiskComponents) {
+            // Only deactivate non shared components (So components are not de-activated twice)
+            if (!diskComponents.contains(c)) {
+                LSMBTreeDiskComponent component = (LSMBTreeDiskComponent) c;
+                BTree btree = component.getBTree();
+                BloomFilter bloomFilter = component.getBloomFilter();
+                btree.deactivate();
+                bloomFilter.deactivate();
+            }
+        }
+        isActivated = false;
+    }
+
+    // The clear method is not used anywhere in AsterixDB! we override it anyway
+    // to exit components first and deal with the two lists
+    @Override
+    public void clear() throws HyracksDataException {
+        if (!isActivated) {
+            throw new HyracksDataException("Failed to clear the index since it is not activated.");
+        }
+        ((ExternalIndexHarness) lsmHarness).indexClear();
+
+        for (ILSMComponent c : diskComponents) {
+            LSMBTreeDiskComponent component = (LSMBTreeDiskComponent) c;
+            component.getBloomFilter().deactivate();
+            component.getBTree().deactivate();
+            component.getBloomFilter().destroy();
+            component.getBTree().destroy();
+            // Remove from second list to avoid destroying twice
+            secondDiskComponents.remove(c);
+        }
+
+        for (ILSMComponent c : secondDiskComponents) {
+            LSMBTreeDiskComponent component = (LSMBTreeDiskComponent) c;
+            component.getBloomFilter().deactivate();
+            component.getBTree().deactivate();
+            component.getBloomFilter().destroy();
+            component.getBTree().destroy();
+        }
+        diskComponents.clear();
+        secondDiskComponents.clear();
+        version = -1;
+    }
+
+    @Override
+    public void destroy() throws HyracksDataException {
+        if (isActivated) {
+            throw new HyracksDataException("Failed to destroy the index since it is activated.");
+        }
+        for (ILSMComponent c : diskComponents) {
+            LSMBTreeDiskComponent component = (LSMBTreeDiskComponent) c;
+            component.getBTree().destroy();
+            component.getBloomFilter().destroy();
+            // Remove from second list to avoid destroying twice
+            secondDiskComponents.remove(c);
+        }
+        for (ILSMComponent c : secondDiskComponents) {
+            LSMBTreeDiskComponent component = (LSMBTreeDiskComponent) c;
+            component.getBTree().destroy();
+            component.getBloomFilter().destroy();
+        }
+        diskComponents.clear();
+        secondDiskComponents.clear();
+        fileManager.deleteDirs();
+        version = -1;
+    }
+
+    @Override
+    public void validate() throws HyracksDataException {
+        for (ILSMComponent c : diskComponents) {
+            BTree btree = (BTree) ((LSMBTreeDiskComponent) c).getBTree();
+            btree.validate();
+        }
+        for (ILSMComponent c : secondDiskComponents) {
+            if (!diskComponents.contains(c)) {
+                BTree btree = (BTree) ((LSMBTreeDiskComponent) c).getBTree();
+                btree.validate();
+            }
+        }
+    }
+
+    // Not supported
+    @Override
+    public void modify(IIndexOperationContext ictx, ITupleReference tuple) throws HyracksDataException, IndexException {
+        throw new UnsupportedOperationException("tuple modify not supported in LSM-Disk-Only-BTree");
+    }
+
+    // Not supported
+    @Override
+    public void scheduleFlush(ILSMIndexOperationContext ctx, ILSMIOOperationCallback callback)
+            throws HyracksDataException {
+        throw new UnsupportedOperationException("flush not supported in LSM-Disk-Only-BTree");
+    }
+
+    // Not supported
+    @Override
+    public ILSMComponent flush(ILSMIOOperation operation) throws HyracksDataException, IndexException {
+        throw new UnsupportedOperationException("flush not supported in LSM-Disk-Only-BTree");
+    }
+
+    // Only support search and merge operations
+    @Override
+    public void getOperationalComponents(ILSMIndexOperationContext ctx) {
+        List<ILSMComponent> operationalComponents = ctx.getComponentHolder();
+        List<ILSMComponent> immutableComponents;
+        // Identify current list in case of a merge
+        if (version == 0) {
+            immutableComponents = diskComponents;
+        } else {
+            immutableComponents = secondDiskComponents;
+        }
+        ExternalBTreeOpContext opCtx = (ExternalBTreeOpContext) ctx;
+        operationalComponents.clear();
+        switch (ctx.getOperation()) {
+            case SEARCH:
+                if (opCtx.getTargetIndexVersion() == 0) {
+                    operationalComponents.addAll(diskComponents);
+                } else {
+                    operationalComponents.addAll(secondDiskComponents);
+                }
+                break;
+            case MERGE:
+                operationalComponents.addAll(ctx.getComponentsToBeMerged());
+                break;
+            case FULL_MERGE:
+                operationalComponents.addAll(immutableComponents);
+                break;
+            default:
+                throw new UnsupportedOperationException("Operation " + ctx.getOperation() + " not supported.");
+        }
+    }
+
+    // For initial load
+    @Override
+    public IIndexBulkLoader createBulkLoader(float fillLevel, boolean verifyInput, long numElementsHint,
+            boolean checkIfEmptyIndex) throws TreeIndexException {
+        try {
+            return new LSMTwoPCBTreeBulkLoader(fillLevel, verifyInput, numElementsHint, checkIfEmptyIndex, false);
+        } catch (HyracksDataException e) {
+            throw new TreeIndexException(e);
+        }
+    }
+
+    // For transaction bulk load <- could consolidate with the above method ->
+    @Override
+    public IIndexBulkLoader createTransactionBulkLoader(float fillLevel, boolean verifyInput, long numElementsHint,
+            boolean checkIfEmptyIndex) throws TreeIndexException {
+        try {
+            return new LSMTwoPCBTreeBulkLoader(fillLevel, verifyInput, numElementsHint, checkIfEmptyIndex, true);
+        } catch (HyracksDataException e) {
+            throw new TreeIndexException(e);
+        }
+    }
+
+    // The bulk loader used for both initial loading and transaction
+    // modifications
+    public class LSMTwoPCBTreeBulkLoader implements IIndexBulkLoader, ITwoPCIndexBulkLoader {
+        private final ILSMComponent component;
+        private final BTreeBulkLoader bulkLoader;
+        private final IIndexBulkLoader builder;
+        private boolean cleanedUpArtifacts = false;
+        private boolean isEmptyComponent = true;
+        private boolean endedBloomFilterLoad = false;
+        private boolean isTransaction;
+        private ITreeIndexTupleWriterFactory frameTupleWriterFactory;
+
+        public LSMTwoPCBTreeBulkLoader(float fillFactor, boolean verifyInput, long numElementsHint,
+                boolean checkIfEmptyIndex, boolean isTransaction) throws TreeIndexException, HyracksDataException {
+            this.isTransaction = isTransaction;
+            // Create the appropriate target
+            if (isTransaction) {
+                try {
+                    component = createTransactionTarget();
+                } catch (HyracksDataException | IndexException e) {
+                    throw new TreeIndexException(e);
+                }
+            } else {
+                if (checkIfEmptyIndex && !isEmptyIndex()) {
+                    throw new TreeIndexException("Cannot load an index that is not empty");
+                }
+                try {
+                    component = createBulkLoadTarget();
+                } catch (HyracksDataException | IndexException e) {
+                    throw new TreeIndexException(e);
+                }
+            }
+
+            frameTupleWriterFactory = ((LSMBTreeDiskComponent) component).getBTree().getLeafFrameFactory()
+                    .getTupleWriterFactory();
+            bulkLoader = (BTreeBulkLoader) ((LSMBTreeDiskComponent) component).getBTree().createBulkLoader(fillFactor,
+                    verifyInput, numElementsHint, false);
+
+            int maxBucketsPerElement = BloomCalculations.maxBucketsPerElement(numElementsHint);
+            BloomFilterSpecification bloomFilterSpec = BloomCalculations.computeBloomSpec(maxBucketsPerElement,
+                    bloomFilterFalsePositiveRate);
+            builder = ((LSMBTreeDiskComponent) component).getBloomFilter().createBuilder(numElementsHint,
+                    bloomFilterSpec.getNumHashes(), bloomFilterSpec.getNumBucketsPerElements());
+        }
+
+        // It is expected that the mode was set to insert operation before
+        // calling add
+        @Override
+        public void add(ITupleReference tuple) throws IndexException, HyracksDataException {
+            try {
+                bulkLoader.add(tuple);
+                builder.add(tuple);
+            } catch (IndexException | HyracksDataException | RuntimeException e) {
+                cleanupArtifacts();
+                throw e;
+            }
+            if (isEmptyComponent) {
+                isEmptyComponent = false;
+            }
+        }
+
+        // This is made public in case of a failure, it is better to delete all
+        // created artifacts.
+        public void cleanupArtifacts() throws HyracksDataException, IndexException {
+            if (!cleanedUpArtifacts) {
+                cleanedUpArtifacts = true;
+                // We make sure to end the bloom filter load to release latches.
+                if (!endedBloomFilterLoad) {
+                    builder.end();
+                    endedBloomFilterLoad = true;
+                }
+                try {
+                    ((LSMBTreeDiskComponent) component).getBTree().deactivate();
+                } catch (HyracksDataException e) {
+                    // Do nothing.. this could've bee
+                }
+                ((LSMBTreeDiskComponent) component).getBTree().destroy();
+                try {
+                    ((LSMBTreeDiskComponent) component).getBloomFilter().deactivate();
+                } catch (HyracksDataException e) {
+                    // Do nothing.. this could've bee
+                }
+                ((LSMBTreeDiskComponent) component).getBloomFilter().destroy();
+            }
+        }
+
+        @Override
+        public void end() throws HyracksDataException, IndexException {
+            if (!cleanedUpArtifacts) {
+                if (!endedBloomFilterLoad) {
+                    builder.end();
+                    endedBloomFilterLoad = true;
+                }
+                bulkLoader.end();
+                if (isEmptyComponent) {
+                    cleanupArtifacts();
+                } else if (isTransaction) {
+                    // Since this is a transaction component, validate and
+                    // deactivate. it could later be added or deleted
+                    markAsValid(component);
+                    BTree btree = ((LSMBTreeDiskComponent) component).getBTree();
+                    BloomFilter bloomFilter = ((LSMBTreeDiskComponent) component).getBloomFilter();
+                    btree.deactivate();
+                    bloomFilter.deactivate();
+                } else {
+                    lsmHarness.addBulkLoadedComponent(component);
+                }
+            }
+        }
+
+        // It is expected that the mode was set to delete operation before
+        // calling delete
+        @Override
+        public void delete(ITupleReference tuple) throws IndexException, HyracksDataException {
+            ((LSMBTreeRefrencingTupleWriterFactory) frameTupleWriterFactory).setMode(IndexOperation.DELETE);
+            try {
+                bulkLoader.add(tuple);
+                builder.add(tuple);
+            } catch (IndexException | HyracksDataException | RuntimeException e) {
+                cleanupArtifacts();
+                throw e;
+            }
+            if (isEmptyComponent) {
+                isEmptyComponent = false;
+            }
+            ((LSMBTreeRefrencingTupleWriterFactory) frameTupleWriterFactory).setMode(IndexOperation.INSERT);
+        }
+
+        @Override
+        public void abort() {
+            try{
+                cleanupArtifacts();
+            } catch(Exception e){
+                // Do nothing
+            }
+        }
+    }
+
+    @Override
+    public String toString() {
+        return "LSMTwoPCBTree [" + fileManager.getBaseDir() + "]";
+    }
+
+    // The accessor for disk only indexes don't use modification callback and always carry the target index version with them
+    @Override
+    public ILSMIndexAccessorInternal createAccessor(IModificationOperationCallback modificationCallback,
+            ISearchOperationCallback searchCallback) {
+        return new LSMBTreeAccessor(lsmHarness, createOpContext(searchCallback, version));
+    }
+
+    // This method creates the appropriate opContext for the targeted version
+    public ExternalBTreeOpContext createOpContext(ISearchOperationCallback searchCallback, int targetVersion) {
+        return new ExternalBTreeOpContext(insertLeafFrameFactory, deleteLeafFrameFactory, searchCallback,
+                componentFactory.getBloomFilterKeyFields().length, cmpFactories, targetVersion);
+    }
+
+    @Override
+    public ILSMIndexAccessorInternal createAccessor(ISearchOperationCallback searchCallback, int targetIndexVersion)
+            throws HyracksDataException {
+        return new LSMBTreeAccessor(lsmHarness, createOpContext(searchCallback, targetIndexVersion));
+    }
+
+    @Override
+    public ITreeIndexFrameFactory getInteriorFrameFactory() {
+        return interiorFrameFactory;
+    }
+
+    @Override
+    public int getFieldCount() {
+        return cmpFactories.length;
+    }
+
+    @Override
+    public int getFileId() {
+        return -1;
+    }
+
+    @Override
+    public IFreePageManager getFreePageManager() {
+        return null;
+    }
+
+    @Override
+    public ITreeIndexFrameFactory getLeafFrameFactory() {
+        return insertLeafFrameFactory;
+    }
+
+    public int getCurrentVersion() {
+        return version;
+    }
+
+    public List<ILSMComponent> getFirstComponentList() {
+        return diskComponents;
+    }
+
+    public List<ILSMComponent> getSecondComponentList() {
+        return secondDiskComponents;
+    }
+
+    @Override
+    public void commitTransaction() throws TreeIndexException, HyracksDataException, IndexException {
+        LSMComponentFileReferences componentFileRefrences = fileManager.getTransactionFileReferenceForCommit();
+        LSMBTreeDiskComponent component = null;
+        if (componentFileRefrences != null) {
+            component = createDiskComponent(componentFactory, componentFileRefrences.getInsertIndexFileReference(),
+                    componentFileRefrences.getBloomFilterFileReference(), false);
+        }
+        ((ExternalIndexHarness) lsmHarness).addTransactionComponents(component);
+    }
+
+    @Override
+    public void abortTransaction() throws TreeIndexException {
+        try {
+            fileManager.deleteTransactionFiles();
+        } catch (HyracksDataException e) {
+            throw new TreeIndexException(e);
+        }
+    }
+
+    @Override
+    public void recoverTransaction() throws TreeIndexException {
+        try {
+            fileManager.recoverTransaction();
+        } catch (HyracksDataException e) {
+            throw new TreeIndexException(e);
+        }
+    }
+    
+    @Override
+    public boolean hasMemoryComponents(){
+        return false;
+    }
+}
diff --git a/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/impls/ExternalBTreeOpContext.java b/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/impls/ExternalBTreeOpContext.java
new file mode 100644
index 0000000..220943f
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/impls/ExternalBTreeOpContext.java
@@ -0,0 +1,121 @@
+/*
+ * 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.hyracks.storage.am.lsm.btree.impls;
+
+import java.util.LinkedList;
+import java.util.List;
+
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import edu.uci.ics.hyracks.storage.am.btree.api.IBTreeLeafFrame;
+import edu.uci.ics.hyracks.storage.am.common.api.IModificationOperationCallback;
+import edu.uci.ics.hyracks.storage.am.common.api.ISearchOperationCallback;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
+import edu.uci.ics.hyracks.storage.am.common.ophelpers.IndexOperation;
+import edu.uci.ics.hyracks.storage.am.common.ophelpers.MultiComparator;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMComponent;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndexOperationContext;
+
+public class ExternalBTreeOpContext implements ILSMIndexOperationContext {
+	public ITreeIndexFrameFactory insertLeafFrameFactory;
+	public ITreeIndexFrameFactory deleteLeafFrameFactory;
+	public IBTreeLeafFrame insertLeafFrame;
+	public IBTreeLeafFrame deleteLeafFrame;
+	public IndexOperation op;
+	public final MultiComparator cmp;
+	public final MultiComparator bloomFilterCmp;
+	public final ISearchOperationCallback searchCallback;
+	private final List<ILSMComponent> componentHolder;
+	private final List<ILSMComponent> componentsToBeMerged;
+	private final int targetIndexVersion;
+
+	public ExternalBTreeOpContext(
+			ITreeIndexFrameFactory insertLeafFrameFactory,
+			ITreeIndexFrameFactory deleteLeafFrameFactory,
+			ISearchOperationCallback searchCallback,
+			int numBloomFilterKeyFields,
+			IBinaryComparatorFactory[] cmpFactories, int targetIndexVersion) {
+		if (cmpFactories != null) {
+			this.cmp = MultiComparator.create(cmpFactories);
+		} else {
+			this.cmp = null;
+		}
+		bloomFilterCmp = MultiComparator.create(cmpFactories, 0,
+				numBloomFilterKeyFields);
+		this.insertLeafFrameFactory = insertLeafFrameFactory;
+		this.deleteLeafFrameFactory = deleteLeafFrameFactory;
+		this.insertLeafFrame = (IBTreeLeafFrame) insertLeafFrameFactory
+				.createFrame();
+		this.deleteLeafFrame = (IBTreeLeafFrame) deleteLeafFrameFactory
+				.createFrame();
+		if (insertLeafFrame != null && this.cmp != null) {
+			insertLeafFrame.setMultiComparator(cmp);
+		}
+		if (deleteLeafFrame != null && this.cmp != null) {
+			deleteLeafFrame.setMultiComparator(cmp);
+		}
+		this.componentHolder = new LinkedList<ILSMComponent>();
+		this.componentsToBeMerged = new LinkedList<ILSMComponent>();
+		this.searchCallback = searchCallback;
+		this.targetIndexVersion = targetIndexVersion;
+	}
+
+	@Override
+	public void setOperation(IndexOperation newOp) {
+		reset();
+		this.op = newOp;
+	}
+
+	@Override
+	public void reset() {
+		componentHolder.clear();
+		componentsToBeMerged.clear();
+	}
+
+	public IndexOperation getOperation() {
+		return op;
+	}
+
+	@Override
+	public List<ILSMComponent> getComponentHolder() {
+		return componentHolder;
+	}
+
+	@Override
+	public ISearchOperationCallback getSearchOperationCallback() {
+		return searchCallback;
+	}
+
+	// Disk only index should never needs a modification callback
+	@Override
+	public IModificationOperationCallback getModificationCallback() {
+		return null;
+	}
+
+	@Override
+	public void setCurrentMutableComponentId(int currentMutableComponentId) {
+		// Do nothing: this method should never be called for this class
+	}
+
+	@Override
+	public List<ILSMComponent> getComponentsToBeMerged() {
+		return componentsToBeMerged;
+	}
+
+	// Used by indexes with global transaction
+	public int getTargetIndexVersion() {
+		return targetIndexVersion;
+	}
+
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/impls/ExternalBTreeWithBuddy.java b/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/impls/ExternalBTreeWithBuddy.java
new file mode 100644
index 0000000..f5fac99
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/impls/ExternalBTreeWithBuddy.java
@@ -0,0 +1,873 @@
+/*
+ * 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.hyracks.storage.am.lsm.btree.impls;
+
+import java.io.IOException;
+import java.util.LinkedList;
+import java.util.List;
+
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.io.FileReference;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
+import edu.uci.ics.hyracks.storage.am.bloomfilter.impls.BloomCalculations;
+import edu.uci.ics.hyracks.storage.am.bloomfilter.impls.BloomFilter;
+import edu.uci.ics.hyracks.storage.am.bloomfilter.impls.BloomFilterFactory;
+import edu.uci.ics.hyracks.storage.am.bloomfilter.impls.BloomFilterSpecification;
+import edu.uci.ics.hyracks.storage.am.btree.impls.BTree;
+import edu.uci.ics.hyracks.storage.am.btree.impls.RangePredicate;
+import edu.uci.ics.hyracks.storage.am.btree.impls.BTree.BTreeBulkLoader;
+import edu.uci.ics.hyracks.storage.am.common.api.IFreePageManager;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexBulkLoader;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexCursor;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexOperationContext;
+import edu.uci.ics.hyracks.storage.am.common.api.IModificationOperationCallback;
+import edu.uci.ics.hyracks.storage.am.common.api.ISearchOperationCallback;
+import edu.uci.ics.hyracks.storage.am.common.api.ISearchPredicate;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexCursor;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
+import edu.uci.ics.hyracks.storage.am.common.api.ITwoPCIndexBulkLoader;
+import edu.uci.ics.hyracks.storage.am.common.api.IndexException;
+import edu.uci.ics.hyracks.storage.am.common.api.TreeIndexException;
+import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallback;
+import edu.uci.ics.hyracks.storage.am.common.ophelpers.IndexOperation;
+import edu.uci.ics.hyracks.storage.am.common.ophelpers.MultiComparator;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ITwoPCIndex;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMComponent;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMComponentFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMHarness;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndexAccessorInternal;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndexFileManager;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndexOperationContext;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMMergePolicy;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTracker;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.AbstractLSMIndex;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.BlockingIOOperationCallbackWrapper;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.LSMOperationType;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.ExternalIndexHarness;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.LSMComponentFileReferences;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.LSMTreeIndexAccessor;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.TreeIndexFactory;
+import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
+import edu.uci.ics.hyracks.storage.common.file.IFileMapProvider;
+
+public class ExternalBTreeWithBuddy extends AbstractLSMIndex implements ITreeIndex, ITwoPCIndex {
+
+    // For creating merge disk components
+    private final LSMBTreeWithBuddyDiskComponentFactory componentFactory;
+
+    private final LSMBTreeWithBuddyDiskComponentFactory bulkComponentFactory;
+
+    private final IBinaryComparatorFactory[] btreeCmpFactories;
+    private final IBinaryComparatorFactory[] buddyBtreeCmpFactories;
+    private int[] buddyBTreeFields;
+
+    // Common for in-memory and on-disk components.
+    private final ITreeIndexFrameFactory btreeInteriorFrameFactory;
+    private final ITreeIndexFrameFactory btreeLeafFrameFactory;
+    private final ITreeIndexFrameFactory buddyBtreeLeafFrameFactory;
+
+    // A second disk component list that will be used when a transaction is
+    // committed and will be seen by subsequent accessors
+    private List<ILSMComponent> secondDiskComponents;
+    private int version = -1;
+
+    public ExternalBTreeWithBuddy(ITreeIndexFrameFactory btreeInteriorFrameFactory,
+            ITreeIndexFrameFactory btreeLeafFrameFactory, ITreeIndexFrameFactory buddyBtreeLeafFrameFactory,
+            IBufferCache diskBufferCache, ILSMIndexFileManager fileManager,
+            TreeIndexFactory<BTree> bulkLoadBTreeFactory, TreeIndexFactory<BTree> copyBtreeFactory,
+            TreeIndexFactory<BTree> buddyBtreeFactory, BloomFilterFactory bloomFilterFactory,
+            IFileMapProvider diskFileMapProvider, double bloomFilterFalsePositiveRate, ILSMMergePolicy mergePolicy,
+            ILSMOperationTracker opTracker, ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallback ioOpCallback,
+            IBinaryComparatorFactory[] btreeCmpFactories, IBinaryComparatorFactory[] buddyBtreeCmpFactories,
+            int[] buddyBTreeFields, int version) {
+        super(diskBufferCache, fileManager, diskFileMapProvider, bloomFilterFalsePositiveRate, mergePolicy, opTracker,
+                ioScheduler, ioOpCallback);
+        this.btreeCmpFactories = btreeCmpFactories;
+        this.buddyBtreeCmpFactories = buddyBtreeCmpFactories;
+        this.buddyBTreeFields = buddyBTreeFields;
+        this.btreeInteriorFrameFactory = btreeInteriorFrameFactory;
+        this.btreeLeafFrameFactory = btreeLeafFrameFactory;
+        this.buddyBtreeLeafFrameFactory = buddyBtreeLeafFrameFactory;
+        this.componentFactory = new LSMBTreeWithBuddyDiskComponentFactory(copyBtreeFactory, buddyBtreeFactory,
+                bloomFilterFactory);
+        this.bulkComponentFactory = new LSMBTreeWithBuddyDiskComponentFactory(bulkLoadBTreeFactory, buddyBtreeFactory,
+                bloomFilterFactory);
+        this.secondDiskComponents = new LinkedList<ILSMComponent>();
+        this.version = version;
+    }
+
+    @Override
+    public void create() throws HyracksDataException {
+        if (isActivated) {
+            throw new HyracksDataException("Failed to create the index since it is activated.");
+        }
+        fileManager.deleteDirs();
+        fileManager.createDirs();
+        diskComponents.clear();
+        secondDiskComponents.clear();
+    }
+
+    @Override
+    public void activate() throws HyracksDataException {
+        if (isActivated) {
+            throw new HyracksDataException("Failed to activate the index since it is already activated.");
+        }
+
+        if (diskComponents.size() == 0 && secondDiskComponents.size() == 0) {
+            //First time activation
+            List<LSMComponentFileReferences> validFileReferences;
+            try {
+                validFileReferences = fileManager.cleanupAndGetValidFiles();
+            } catch (IndexException e) {
+                throw new HyracksDataException(e);
+            }
+            for (LSMComponentFileReferences lsmComonentFileReference : validFileReferences) {
+                LSMBTreeWithBuddyDiskComponent component;
+                try {
+                    component = createDiskComponent(componentFactory,
+                            lsmComonentFileReference.getInsertIndexFileReference(),
+                            lsmComonentFileReference.getDeleteIndexFileReference(),
+                            lsmComonentFileReference.getBloomFilterFileReference(), false);
+                } catch (IndexException e) {
+                    throw new HyracksDataException(e);
+                }
+                diskComponents.add(component);
+                secondDiskComponents.add(component);
+            }
+            ((ExternalIndexHarness) lsmHarness).indexFirstTimeActivated();
+        } else {
+            // This index has been opened before or is brand new with no
+            // components. It should also maintain the version pointer
+            for (ILSMComponent c : diskComponents) {
+                LSMBTreeWithBuddyDiskComponent component = (LSMBTreeWithBuddyDiskComponent) c;
+                BTree btree = component.getBTree();
+                BTree buddyBtree = component.getBuddyBTree();
+                BloomFilter bloomFilter = component.getBloomFilter();
+                btree.activate();
+                buddyBtree.activate();
+                bloomFilter.activate();
+            }
+            for (ILSMComponent c : secondDiskComponents) {
+                // Only activate non shared components
+                if (!diskComponents.contains(c)) {
+                    LSMBTreeWithBuddyDiskComponent component = (LSMBTreeWithBuddyDiskComponent) c;
+                    BTree btree = component.getBTree();
+                    BTree buddyBtree = component.getBuddyBTree();
+                    BloomFilter bloomFilter = component.getBloomFilter();
+                    btree.activate();
+                    buddyBtree.activate();
+                    bloomFilter.activate();
+                }
+            }
+        }
+        isActivated = true;
+    }
+
+    @Override
+    public void clear() throws HyracksDataException {
+        if (!isActivated) {
+            throw new HyracksDataException("Failed to clear the index since it is not activated.");
+        }
+        ((ExternalIndexHarness) lsmHarness).indexClear();
+
+        for (ILSMComponent c : diskComponents) {
+            LSMBTreeWithBuddyDiskComponent component = (LSMBTreeWithBuddyDiskComponent) c;
+            component.getBTree().deactivate();
+            component.getBuddyBTree().deactivate();
+            component.getBloomFilter().deactivate();
+            component.getBTree().destroy();
+            component.getBloomFilter().destroy();
+            component.getBuddyBTree().destroy();
+            // Remove from second list to avoid destroying twice
+            secondDiskComponents.remove(c);
+        }
+
+        for (ILSMComponent c : secondDiskComponents) {
+            LSMBTreeWithBuddyDiskComponent component = (LSMBTreeWithBuddyDiskComponent) c;
+            component.getBTree().deactivate();
+            component.getBloomFilter().deactivate();
+            component.getBuddyBTree().deactivate();
+            component.getBTree().destroy();
+            component.getBloomFilter().destroy();
+            component.getBuddyBTree().destroy();
+        }
+
+        diskComponents.clear();
+        secondDiskComponents.clear();
+        version = -1;
+    }
+
+    @Override
+    public void deactivate() throws HyracksDataException {
+        deactivate(true);
+    }
+
+    @Override
+    public void destroy() throws HyracksDataException {
+        if (isActivated) {
+            throw new HyracksDataException("Failed to destroy the index since it is activated.");
+        }
+        for (ILSMComponent c : diskComponents) {
+            LSMBTreeWithBuddyDiskComponent component = (LSMBTreeWithBuddyDiskComponent) c;
+            component.getBTree().destroy();
+            component.getBuddyBTree().destroy();
+            component.getBloomFilter().destroy();
+            // Remove from second list to avoid destroying twice
+            secondDiskComponents.remove(c);
+        }
+        for (ILSMComponent c : secondDiskComponents) {
+            LSMBTreeWithBuddyDiskComponent component = (LSMBTreeWithBuddyDiskComponent) c;
+            component.getBTree().destroy();
+            component.getBuddyBTree().destroy();
+            component.getBloomFilter().destroy();
+        }
+        diskComponents.clear();
+        secondDiskComponents.clear();
+        fileManager.deleteDirs();
+        version = -1;
+    }
+
+    @Override
+    public ILSMIndexAccessorInternal createAccessor(IModificationOperationCallback modificationCallback,
+            ISearchOperationCallback searchCallback) throws HyracksDataException {
+        return new LSMBTreeWithBuddyAccessor(lsmHarness, createOpContext(searchCallback, version));
+    }
+
+    @Override
+    public void validate() throws HyracksDataException {
+        throw new UnsupportedOperationException("Validation not implemented for LSM B-Trees with Buddy B-Tree.");
+    }
+
+    @Override
+    public long getMemoryAllocationSize() {
+        return 0;
+    }
+
+    // The subsume merged components is overridden to account for:
+    // Maintaining two versions of the index
+    @Override
+    public void subsumeMergedComponents(ILSMComponent newComponent, List<ILSMComponent> mergedComponents)
+            throws HyracksDataException {
+        List<ILSMComponent> newerList;
+        List<ILSMComponent> olderList;
+        if (version == 0) {
+            newerList = diskComponents;
+            olderList = secondDiskComponents;
+        } else {
+            newerList = secondDiskComponents;
+            olderList = diskComponents;
+        }
+
+        // Check if merge will affect the older list
+        if (olderList.containsAll(mergedComponents)) {
+            int swapIndex = olderList.indexOf(mergedComponents.get(0));
+            olderList.removeAll(mergedComponents);
+            olderList.add(swapIndex, newComponent);
+        }
+
+        // The new list will always have all the merged components
+        int swapIndex = newerList.indexOf(mergedComponents.get(0));
+        newerList.removeAll(mergedComponents);
+        newerList.add(swapIndex, newComponent);
+    }
+
+    // For initial load
+    @Override
+    public IIndexBulkLoader createBulkLoader(float fillLevel, boolean verifyInput, long numElementsHint,
+            boolean checkIfEmptyIndex) throws TreeIndexException {
+        try {
+            return new LSMTwoPCBTreeWithBuddyBulkLoader(fillLevel, verifyInput, 0, checkIfEmptyIndex,
+                    false);
+        } catch (HyracksDataException e) {
+            throw new TreeIndexException(e);
+        }
+    }
+
+    // For transaction bulk load <- could consolidate with the above method ->
+    public IIndexBulkLoader createTransactionBulkLoader(float fillLevel, boolean verifyInput, long numElementsHint,
+            boolean checkIfEmptyIndex) throws TreeIndexException {
+        try {
+            return new LSMTwoPCBTreeWithBuddyBulkLoader(fillLevel, verifyInput, numElementsHint, checkIfEmptyIndex,
+                    true);
+        } catch (HyracksDataException e) {
+            throw new TreeIndexException(e);
+        }
+    }
+
+    @Override
+    public void modify(IIndexOperationContext ictx, ITupleReference tuple) throws HyracksDataException, IndexException {
+        throw new UnsupportedOperationException("tuple modify not supported in LSM-Disk-Only-BTree");
+    }
+
+    @Override
+    public void search(ILSMIndexOperationContext ictx, IIndexCursor cursor, ISearchPredicate pred)
+            throws HyracksDataException, IndexException {
+        ExternalBTreeWithBuddyOpContext ctx = (ExternalBTreeWithBuddyOpContext) ictx;
+        List<ILSMComponent> operationalComponents = ictx.getComponentHolder();
+
+        LSMBTreeWithBuddyCursorInitialState initialState = new LSMBTreeWithBuddyCursorInitialState(
+                btreeInteriorFrameFactory, btreeLeafFrameFactory, buddyBtreeLeafFrameFactory, lsmHarness,
+                MultiComparator.create(btreeCmpFactories), MultiComparator.create(buddyBtreeCmpFactories),
+                ctx.searchCallback, operationalComponents);
+
+        cursor.open(initialState, pred);
+    }
+
+    @Override
+    public void scheduleFlush(ILSMIndexOperationContext ctx, ILSMIOOperationCallback callback)
+            throws HyracksDataException {
+        throw new UnsupportedOperationException("flush not supported in LSM-Disk-Only-BTree");
+    }
+
+    @Override
+    public ILSMComponent flush(ILSMIOOperation operation) throws HyracksDataException, IndexException {
+        throw new UnsupportedOperationException("flush not supported in LSM-Disk-Only-BTree");
+    }
+
+    protected LSMComponentFileReferences getMergeTargetFileName(List<ILSMComponent> mergingDiskComponents)
+            throws HyracksDataException {
+        BTree firstTree = ((LSMBTreeWithBuddyDiskComponent) mergingDiskComponents.get(0)).getBTree();
+        BTree lastTree = ((LSMBTreeWithBuddyDiskComponent) mergingDiskComponents.get(mergingDiskComponents.size() - 1))
+                .getBTree();
+        FileReference firstFile = diskFileMapProvider.lookupFileName(firstTree.getFileId());
+        FileReference lastFile = diskFileMapProvider.lookupFileName(lastTree.getFileId());
+        LSMComponentFileReferences fileRefs = fileManager.getRelMergeFileReference(firstFile.getFile().getName(),
+                lastFile.getFile().getName());
+        return fileRefs;
+    }
+
+    @Override
+    public void scheduleMerge(ILSMIndexOperationContext ctx, ILSMIOOperationCallback callback)
+            throws HyracksDataException, IndexException {
+        ILSMIndexOperationContext bctx = createOpContext(NoOpOperationCallback.INSTANCE, 0);
+        bctx.setOperation(IndexOperation.MERGE);
+        List<ILSMComponent> mergingComponents = ctx.getComponentHolder();
+        ITreeIndexCursor cursor = new LSMBTreeWithBuddySortedCursor(bctx, buddyBTreeFields);
+        LSMComponentFileReferences relMergeFileRefs = getMergeTargetFileName(mergingComponents);
+        ILSMIndexAccessorInternal accessor = new LSMBTreeWithBuddyAccessor(lsmHarness, bctx);
+
+        // Since we have two lists of components, to tell whether we need to
+        // keep deleted tuples, we need to know
+        // which list to check against and we need to synchronize for this
+        boolean keepDeleteTuples = false;
+        if (version == 0) {
+            keepDeleteTuples = mergingComponents.get(mergingComponents.size() - 1) != diskComponents.get(diskComponents
+                    .size() - 1);
+        } else {
+            keepDeleteTuples = mergingComponents.get(mergingComponents.size() - 1) != secondDiskComponents
+                    .get(secondDiskComponents.size() - 1);
+        }
+
+        ioScheduler.scheduleOperation(new LSMBTreeWithBuddyMergeOperation((ILSMIndexAccessorInternal) accessor,
+                mergingComponents, cursor, relMergeFileRefs.getInsertIndexFileReference(), relMergeFileRefs
+                        .getDeleteIndexFileReference(), relMergeFileRefs.getBloomFilterFileReference(), callback,
+                fileManager.getBaseDir(), keepDeleteTuples));
+    }
+
+    // This method creates the appropriate opContext for the targeted version
+    public ExternalBTreeWithBuddyOpContext createOpContext(ISearchOperationCallback searchCallback, int targetVersion) {
+        return new ExternalBTreeWithBuddyOpContext(btreeCmpFactories, buddyBtreeCmpFactories, searchCallback,
+                targetVersion);
+    }
+
+    @Override
+    public ILSMComponent merge(ILSMIOOperation operation) throws HyracksDataException, IndexException {
+        LSMBTreeWithBuddyMergeOperation mergeOp = (LSMBTreeWithBuddyMergeOperation) operation;
+        ITreeIndexCursor cursor = mergeOp.getCursor();
+        ISearchPredicate btreeSearchPred = new RangePredicate(null, null, true, true, null, null);
+        ILSMIndexOperationContext opCtx = ((LSMBTreeWithBuddySortedCursor) cursor).getOpCtx();
+        opCtx.getComponentHolder().addAll(mergeOp.getMergingComponents());
+        search(opCtx, cursor, btreeSearchPred);
+
+        LSMBTreeWithBuddyDiskComponent mergedComponent = createDiskComponent(componentFactory,
+                mergeOp.getBTreeMergeTarget(), mergeOp.getBuddyBTreeMergeTarget(), mergeOp.getBloomFilterMergeTarget(),
+                true);
+
+        // In case we must keep the deleted-keys BuddyBTrees, then they must be
+        // merged *before* merging the b-trees so that
+        // lsmHarness.endSearch() is called once when the b-trees have been
+        // merged.
+
+        if (mergeOp.isKeepDeletedTuples()) {
+            // Keep the deleted tuples since the oldest disk component is not
+            // included in the merge operation
+            LSMBuddyBTreeMergeCursor buddyBtreeCursor = new LSMBuddyBTreeMergeCursor(opCtx);
+            search(opCtx, buddyBtreeCursor, btreeSearchPred);
+
+            BTree buddyBtree = mergedComponent.getBuddyBTree();
+            IIndexBulkLoader buddyBtreeBulkLoader = buddyBtree.createBulkLoader(1.0f, true, 0L, false);
+
+            long numElements = 0L;
+            for (int i = 0; i < mergeOp.getMergingComponents().size(); ++i) {
+                numElements += ((LSMBTreeWithBuddyDiskComponent) mergeOp.getMergingComponents().get(i))
+                        .getBloomFilter().getNumElements();
+            }
+
+            int maxBucketsPerElement = BloomCalculations.maxBucketsPerElement(numElements);
+            BloomFilterSpecification bloomFilterSpec = BloomCalculations.computeBloomSpec(maxBucketsPerElement,
+                    bloomFilterFalsePositiveRate);
+            IIndexBulkLoader builder = mergedComponent.getBloomFilter().createBuilder(numElements,
+                    bloomFilterSpec.getNumHashes(), bloomFilterSpec.getNumBucketsPerElements());
+
+            try {
+                while (buddyBtreeCursor.hasNext()) {
+                    buddyBtreeCursor.next();
+                    ITupleReference tuple = buddyBtreeCursor.getTuple();
+                    buddyBtreeBulkLoader.add(tuple);
+                    builder.add(tuple);
+                }
+            } finally {
+                buddyBtreeCursor.close();
+                builder.end();
+            }
+            buddyBtreeBulkLoader.end();
+        }
+
+        IIndexBulkLoader bulkLoader = mergedComponent.getBTree().createBulkLoader(1.0f, false, 0L, false);
+        try {
+            while (cursor.hasNext()) {
+                cursor.next();
+                ITupleReference frameTuple = cursor.getTuple();
+                bulkLoader.add(frameTuple);
+            }
+        } finally {
+            cursor.close();
+        }
+        bulkLoader.end();
+        return mergedComponent;
+    }
+
+    @Override
+    public void getOperationalComponents(ILSMIndexOperationContext ctx) {
+        List<ILSMComponent> operationalComponents = ctx.getComponentHolder();
+        List<ILSMComponent> immutableComponents;
+        // Identify current list in case of a merge
+        if (version == 0) {
+            immutableComponents = diskComponents;
+        } else {
+            immutableComponents = secondDiskComponents;
+        }
+
+        ExternalBTreeWithBuddyOpContext opCtx = (ExternalBTreeWithBuddyOpContext) ctx;
+        operationalComponents.clear();
+        switch (ctx.getOperation()) {
+            case SEARCH:
+                if (opCtx.getTargetIndexVersion() == 0) {
+                    operationalComponents.addAll(diskComponents);
+                } else {
+                    operationalComponents.addAll(secondDiskComponents);
+                }
+                break;
+            case MERGE:
+                operationalComponents.addAll(ctx.getComponentsToBeMerged());
+                break;
+            case FULL_MERGE:
+                operationalComponents.addAll(immutableComponents);
+                break;
+            case FLUSH:
+                // Do nothing. this is left here even though the index never
+                // performs flushes because a flush is triggered by
+                // dataset lifecycle manager when closing an index. Having no
+                // components is a no operation
+                break;
+            default:
+                throw new UnsupportedOperationException("Operation " + ctx.getOperation() + " not supported.");
+        }
+    }
+
+    @Override
+    public void markAsValid(ILSMComponent lsmComponent) throws HyracksDataException {
+        LSMBTreeWithBuddyDiskComponent component = (LSMBTreeWithBuddyDiskComponent) lsmComponent;
+        // Flush the bloom filter first.
+        int fileId = component.getBloomFilter().getFileId();
+        IBufferCache bufferCache = component.getBTree().getBufferCache();
+        int startPage = 0;
+        int maxPage = component.getBloomFilter().getNumPages();
+        forceFlushDirtyPages(bufferCache, fileId, startPage, maxPage);
+        forceFlushDirtyPages(component.getBTree());
+        markAsValidInternal(component.getBTree());
+        forceFlushDirtyPages(component.getBuddyBTree());
+        markAsValidInternal(component.getBuddyBTree());
+    }
+
+    // This function is used when a new component is to be committed -- is
+    // called by the harness.
+    public void commitTransactionDiskComponent(ILSMComponent newComponent) throws HyracksDataException {
+
+        // determine which list is the new one and flip the pointer
+        List<ILSMComponent> newerList;
+        List<ILSMComponent> olderList;
+        if (version == 0) {
+            newerList = diskComponents;
+            olderList = secondDiskComponents;
+            version = 1;
+        } else {
+            newerList = secondDiskComponents;
+            olderList = diskComponents;
+            version = 0;
+        }
+        // Remove components from list
+        olderList.clear();
+        // Add components
+        olderList.addAll(newerList);
+        if (newComponent != null) {
+            // Add it to the list
+            olderList.add(0, newComponent);
+        }
+    }
+
+    @Override
+    public void deactivate(boolean flushOnExit) throws HyracksDataException {
+        if (!isActivated) {
+            throw new HyracksDataException("Failed to deactivate the index since it is already deactivated.");
+        }
+
+        if (flushOnExit) {
+            BlockingIOOperationCallbackWrapper cb = new BlockingIOOperationCallbackWrapper(ioOpCallback);
+            cb.afterFinalize(LSMOperationType.FLUSH, null);
+        }
+        // Even though, we deactivate the index, we don't exit components or
+        // modify any of the lists to make sure they
+        // are there if the index was opened again
+
+        for (ILSMComponent c : diskComponents) {
+            LSMBTreeWithBuddyDiskComponent component = (LSMBTreeWithBuddyDiskComponent) c;
+            BTree btree = component.getBTree();
+            BTree buddyBtree = component.getBuddyBTree();
+            BloomFilter bloomFilter = component.getBloomFilter();
+            btree.deactivate();
+            buddyBtree.deactivate();
+            bloomFilter.deactivate();
+        }
+        for (ILSMComponent c : secondDiskComponents) {
+            // Only deactivate non shared components
+            if (!diskComponents.contains(c)) {
+                LSMBTreeWithBuddyDiskComponent component = (LSMBTreeWithBuddyDiskComponent) c;
+                BTree btree = component.getBTree();
+                BTree buddyBtree = component.getBuddyBTree();
+                BloomFilter bloomFilter = component.getBloomFilter();
+                btree.deactivate();
+                buddyBtree.deactivate();
+                bloomFilter.deactivate();
+            }
+        }
+        isActivated = false;
+    }
+
+    @Override
+    public ITreeIndexFrameFactory getLeafFrameFactory() {
+        return btreeLeafFrameFactory;
+    }
+
+    @Override
+    public ITreeIndexFrameFactory getInteriorFrameFactory() {
+        return btreeInteriorFrameFactory;
+    }
+
+    @Override
+    public IFreePageManager getFreePageManager() {
+        // This method should never be called for disk only indexes
+        return null;
+    }
+
+    @Override
+    public int getFieldCount() {
+        return btreeCmpFactories.length;
+    }
+
+    @Override
+    public int getRootPageId() {
+        // This method should never be called for this index
+        return 0;
+    }
+
+    @Override
+    public int getFileId() {
+        // This method should never be called for this index
+        return 0;
+    }
+
+    @Override
+    public IBinaryComparatorFactory[] getComparatorFactories() {
+        return btreeCmpFactories;
+    }
+
+    private LSMBTreeWithBuddyDiskComponent createDiskComponent(ILSMComponentFactory factory,
+            FileReference insertFileRef, FileReference deleteFileRef, FileReference bloomFilterFileRef,
+            boolean createComponent) throws HyracksDataException, IndexException {
+        // Create new instance.
+        LSMBTreeWithBuddyDiskComponent component = (LSMBTreeWithBuddyDiskComponent) factory
+                .createLSMComponentInstance(new LSMComponentFileReferences(insertFileRef, deleteFileRef,
+                        bloomFilterFileRef));
+        if (createComponent) {
+            component.getBTree().create();
+            component.getBuddyBTree().create();
+            component.getBloomFilter().create();
+        }
+
+        component.getBTree().activate();
+        component.getBuddyBTree().activate();
+        component.getBloomFilter().activate();
+        return component;
+    }
+
+    // even though the index doesn't support record level modification, the
+    // accessor will try to do it
+    // we could throw the exception here but we don't. it will eventually be
+    // thrown by the index itself
+    public class LSMBTreeWithBuddyAccessor extends LSMTreeIndexAccessor {
+        public LSMBTreeWithBuddyAccessor(ILSMHarness lsmHarness, ILSMIndexOperationContext ctx) {
+            super(lsmHarness, ctx);
+        }
+
+        @Override
+        public ITreeIndexCursor createSearchCursor(boolean exclusive) {
+            return new LSMBTreeWithBuddySearchCursor(ctx, buddyBTreeFields);
+        }
+
+        public MultiComparator getBTreeMultiComparator() {
+            ExternalBTreeWithBuddyOpContext concreteCtx = (ExternalBTreeWithBuddyOpContext) ctx;
+            return concreteCtx.getBTreeMultiComparator();
+        }
+
+        public MultiComparator getBodyBTreeMultiComparator() {
+            ExternalBTreeWithBuddyOpContext concreteCtx = (ExternalBTreeWithBuddyOpContext) ctx;
+            return concreteCtx.getBuddyBTreeMultiComparator();
+        }
+    }
+
+    // The bulk loader used for both initial loading and transaction
+    // modifications
+    public class LSMTwoPCBTreeWithBuddyBulkLoader implements IIndexBulkLoader, ITwoPCIndexBulkLoader {
+        private final ILSMComponent component;
+        private final BTreeBulkLoader btreeBulkLoader;
+        private final BTreeBulkLoader buddyBtreeBulkLoader;
+        private final IIndexBulkLoader builder;
+        private boolean cleanedUpArtifacts = false;
+        private boolean isEmptyComponent = true;
+        private boolean endedBloomFilterLoad = false;
+        private boolean isTransaction;
+
+        public LSMTwoPCBTreeWithBuddyBulkLoader(float fillFactor, boolean verifyInput, long numElementsHint,
+                boolean checkIfEmptyIndex, boolean isTransaction) throws TreeIndexException, HyracksDataException {
+            this.isTransaction = isTransaction;
+            // Create the appropriate target
+            if (isTransaction) {
+                try {
+                    component = createTransactionTarget();
+                } catch (HyracksDataException | IndexException e) {
+                    throw new TreeIndexException(e);
+                }
+            } else {
+                if (checkIfEmptyIndex && !isEmptyIndex()) {
+                    throw new TreeIndexException("Cannot load an index that is not empty");
+                }
+                try {
+                    component = createBulkLoadTarget();
+                } catch (HyracksDataException | IndexException e) {
+                    throw new TreeIndexException(e);
+                }
+            }
+
+            // Create the three loaders
+            btreeBulkLoader = (BTreeBulkLoader) ((LSMBTreeWithBuddyDiskComponent) component).getBTree()
+                    .createBulkLoader(fillFactor, verifyInput, numElementsHint, false);
+            buddyBtreeBulkLoader = (BTreeBulkLoader) ((LSMBTreeWithBuddyDiskComponent) component).getBuddyBTree()
+                    .createBulkLoader(fillFactor, verifyInput, numElementsHint, false);
+            int maxBucketsPerElement = BloomCalculations.maxBucketsPerElement(numElementsHint);
+            BloomFilterSpecification bloomFilterSpec = BloomCalculations.computeBloomSpec(maxBucketsPerElement,
+                    bloomFilterFalsePositiveRate);
+            builder = ((LSMBTreeWithBuddyDiskComponent) component).getBloomFilter().createBuilder(numElementsHint,
+                    bloomFilterSpec.getNumHashes(), bloomFilterSpec.getNumBucketsPerElements());
+        }
+
+        @Override
+        public void add(ITupleReference tuple) throws IndexException, HyracksDataException {
+            try {
+                btreeBulkLoader.add(tuple);
+            } catch (IndexException | HyracksDataException | RuntimeException e) {
+                cleanupArtifacts();
+                throw e;
+            }
+            if (isEmptyComponent) {
+                isEmptyComponent = false;
+            }
+        }
+
+        // This is made public in case of a failure, it is better to delete all
+        // created artifacts.
+        public void cleanupArtifacts() throws HyracksDataException {
+            if (!cleanedUpArtifacts) {
+                cleanedUpArtifacts = true;
+                try {
+                    ((LSMBTreeWithBuddyDiskComponent) component).getBTree().deactivate();
+                } catch (Exception e) {
+
+                }
+                ((LSMBTreeWithBuddyDiskComponent) component).getBTree().destroy();
+                try {
+                    ((LSMBTreeWithBuddyDiskComponent) component).getBuddyBTree().deactivate();
+                } catch (Exception e) {
+
+                }
+                ((LSMBTreeWithBuddyDiskComponent) component).getBuddyBTree().destroy();
+                try {
+                    ((LSMBTreeWithBuddyDiskComponent) component).getBloomFilter().deactivate();
+                } catch (Exception e) {
+
+                }
+                ((LSMBTreeWithBuddyDiskComponent) component).getBloomFilter().destroy();
+            }
+        }
+
+        @Override
+        public void end() throws HyracksDataException, IndexException {
+            if (!cleanedUpArtifacts) {
+                if (!endedBloomFilterLoad) {
+                    builder.end();
+                    endedBloomFilterLoad = true;
+                }
+                btreeBulkLoader.end();
+                buddyBtreeBulkLoader.end();
+                if (isEmptyComponent) {
+                    cleanupArtifacts();
+                } else if (isTransaction) {
+                    // Since this is a transaction component, validate and
+                    // deactivate. it could later be added or deleted
+                    markAsValid(component);
+                    BTree btree = ((LSMBTreeWithBuddyDiskComponent) component).getBTree();
+                    BTree buddyBtree = ((LSMBTreeWithBuddyDiskComponent) component).getBuddyBTree();
+                    BloomFilter bloomFilter = ((LSMBTreeWithBuddyDiskComponent) component).getBloomFilter();
+                    btree.deactivate();
+                    buddyBtree.deactivate();
+                    bloomFilter.deactivate();
+                } else {
+                    lsmHarness.addBulkLoadedComponent(component);
+                }
+            }
+        }
+
+        @Override
+        public void delete(ITupleReference tuple) throws IndexException, HyracksDataException {
+            try {
+                buddyBtreeBulkLoader.add(tuple);
+                builder.add(tuple);
+            } catch (IndexException | HyracksDataException | RuntimeException e) {
+                cleanupArtifacts();
+                throw e;
+            }
+            if (isEmptyComponent) {
+                isEmptyComponent = false;
+            }
+        }
+
+        @Override
+        public void abort() {
+            try {
+                cleanupArtifacts();
+            } catch (Exception e) {
+            }
+        }
+    }
+
+    protected ILSMComponent createBulkLoadTarget() throws HyracksDataException, IndexException {
+        LSMComponentFileReferences componentFileRefs = fileManager.getRelFlushFileReference();
+        return createDiskComponent(bulkComponentFactory, componentFileRefs.getInsertIndexFileReference(),
+                componentFileRefs.getDeleteIndexFileReference(), componentFileRefs.getBloomFilterFileReference(), true);
+    }
+
+    // This method is used to create a target for a bulk modify operation. This
+    // component must then eventually be either committed or deleted
+    private ILSMComponent createTransactionTarget() throws HyracksDataException, IndexException {
+        LSMComponentFileReferences componentFileRefs;
+        try {
+            componentFileRefs = fileManager.getNewTransactionFileReference();
+        } catch (IOException e) {
+            throw new HyracksDataException("Failed to create transaction components", e);
+        }
+        return createDiskComponent(bulkComponentFactory, componentFileRefs.getInsertIndexFileReference(),
+                componentFileRefs.getDeleteIndexFileReference(), componentFileRefs.getBloomFilterFileReference(), true);
+    }
+
+    @Override
+    public ILSMIndexAccessorInternal createAccessor(ISearchOperationCallback searchCallback, int targetIndexVersion)
+            throws HyracksDataException {
+        return new LSMBTreeWithBuddyAccessor(lsmHarness, createOpContext(searchCallback, targetIndexVersion));
+    }
+
+    // This function in an instance of this index is only used after a bulk load
+    // is successful
+    // it will therefore add the component to the first list and enter it.
+    @Override
+    public void addComponent(ILSMComponent c) throws HyracksDataException {
+        if (version == 0) {
+            diskComponents.add(0, c);
+        } else if (version == 1) {
+            secondDiskComponents.add(0, c);
+        }
+    }
+
+    public int getCurrentVersion() {
+        return version;
+    }
+
+    public List<ILSMComponent> getFirstComponentList() {
+        return diskComponents;
+    }
+
+    public List<ILSMComponent> getSecondComponentList() {
+        return secondDiskComponents;
+    }
+
+    @Override
+    public void commitTransaction() throws HyracksDataException, IndexException {
+        LSMComponentFileReferences componentFileRefrences = fileManager.getTransactionFileReferenceForCommit();
+        LSMBTreeWithBuddyDiskComponent component = null;
+        if (componentFileRefrences != null) {
+            component = createDiskComponent(componentFactory, componentFileRefrences.getInsertIndexFileReference(),
+                    componentFileRefrences.getDeleteIndexFileReference(),
+                    componentFileRefrences.getBloomFilterFileReference(), false);
+        }
+        ((ExternalIndexHarness) lsmHarness).addTransactionComponents(component);
+    }
+
+    @Override
+    public void abortTransaction() throws TreeIndexException {
+        try {
+            fileManager.deleteTransactionFiles();
+        } catch (HyracksDataException e) {
+            throw new TreeIndexException(e);
+        }
+    }
+
+    @Override
+    public void recoverTransaction() throws TreeIndexException {
+        try {
+            fileManager.recoverTransaction();
+        } catch (HyracksDataException e) {
+            throw new TreeIndexException(e);
+        }
+    }
+    
+    @Override
+    public boolean hasMemoryComponents(){
+        return false;
+    }
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/impls/ExternalBTreeWithBuddyOpContext.java b/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/impls/ExternalBTreeWithBuddyOpContext.java
new file mode 100644
index 0000000..e676bc8
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/impls/ExternalBTreeWithBuddyOpContext.java
@@ -0,0 +1,104 @@
+/*
+ * 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.hyracks.storage.am.lsm.btree.impls;
+
+import java.util.LinkedList;
+import java.util.List;
+
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import edu.uci.ics.hyracks.storage.am.common.api.IModificationOperationCallback;
+import edu.uci.ics.hyracks.storage.am.common.api.ISearchOperationCallback;
+import edu.uci.ics.hyracks.storage.am.common.ophelpers.IndexOperation;
+import edu.uci.ics.hyracks.storage.am.common.ophelpers.MultiComparator;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMComponent;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndexOperationContext;
+
+public class ExternalBTreeWithBuddyOpContext implements
+		ILSMIndexOperationContext {
+	private IndexOperation op;
+	private MultiComparator bTreeCmp;
+	private MultiComparator buddyBTreeCmp;
+	public final List<ILSMComponent> componentHolder;
+	private final List<ILSMComponent> componentsToBeMerged;
+	public final ISearchOperationCallback searchCallback;
+	private final int targetIndexVersion;
+
+	public ExternalBTreeWithBuddyOpContext(
+			IBinaryComparatorFactory[] btreeCmpFactories,
+			IBinaryComparatorFactory[] buddyBtreeCmpFactories,
+			ISearchOperationCallback searchCallback, int targetIndexVersion) {
+
+		this.componentHolder = new LinkedList<ILSMComponent>();
+		this.componentsToBeMerged = new LinkedList<ILSMComponent>();
+		this.searchCallback = searchCallback;
+		this.targetIndexVersion = targetIndexVersion;
+		this.bTreeCmp = MultiComparator.create(btreeCmpFactories);
+		this.buddyBTreeCmp = MultiComparator.create(buddyBtreeCmpFactories);
+	}
+
+	public void setOperation(IndexOperation newOp) {
+		reset();
+		this.op = newOp;
+	}
+
+	@Override
+	public void setCurrentMutableComponentId(int currentMutableComponentId) {
+		// Do nothing. this should never be called for disk only indexes
+	}
+
+	@Override
+	public void reset() {
+		componentHolder.clear();
+		componentsToBeMerged.clear();
+	}
+
+	@Override
+	public IndexOperation getOperation() {
+		return op;
+	}
+
+	public MultiComparator getBTreeMultiComparator() {
+		return bTreeCmp;
+	}
+
+	public MultiComparator getBuddyBTreeMultiComparator() {
+		return buddyBTreeCmp;
+	}
+
+	@Override
+	public List<ILSMComponent> getComponentHolder() {
+		return componentHolder;
+	}
+
+	@Override
+	public ISearchOperationCallback getSearchOperationCallback() {
+		return searchCallback;
+	}
+
+	// This should never be needed for disk only indexes
+	@Override
+	public IModificationOperationCallback getModificationCallback() {
+		return null;
+	}
+
+	@Override
+	public List<ILSMComponent> getComponentsToBeMerged() {
+		return componentsToBeMerged;
+	}
+
+	public int getTargetIndexVersion() {
+		return targetIndexVersion;
+	}
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/impls/LSMBTree.java b/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/impls/LSMBTree.java
index 19307e9..9a0e315 100644
--- a/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/impls/LSMBTree.java
+++ b/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/impls/LSMBTree.java
@@ -75,17 +75,17 @@
 public class LSMBTree extends AbstractLSMIndex implements ITreeIndex {
 
     // For creating BTree's used in flush and merge.
-    private final LSMBTreeDiskComponentFactory componentFactory;
+    protected final LSMBTreeDiskComponentFactory componentFactory;
     // For creating BTree's used in bulk load. Different from diskBTreeFactory
     // because it should have a different tuple writer in it's leaf frames.
-    private final LSMBTreeDiskComponentFactory bulkLoadComponentFactory;
+    protected final LSMBTreeDiskComponentFactory bulkLoadComponentFactory;
 
     // Common for in-memory and on-disk components.
-    private final ITreeIndexFrameFactory insertLeafFrameFactory;
-    private final ITreeIndexFrameFactory deleteLeafFrameFactory;
-    private final IBinaryComparatorFactory[] cmpFactories;
+    protected final ITreeIndexFrameFactory insertLeafFrameFactory;
+    protected final ITreeIndexFrameFactory deleteLeafFrameFactory;
+    protected final IBinaryComparatorFactory[] cmpFactories;
 
-    private final boolean needKeyDupCheck;
+    protected final boolean needKeyDupCheck;
 
     public LSMBTree(List<IVirtualBufferCache> virtualBufferCaches, ITreeIndexFrameFactory interiorFrameFactory,
             ITreeIndexFrameFactory insertLeafFrameFactory, ITreeIndexFrameFactory deleteLeafFrameFactory,
@@ -115,6 +115,24 @@
         this.needKeyDupCheck = needKeyDupCheck;
     }
 
+    // Without memory components
+    public LSMBTree(ITreeIndexFrameFactory interiorFrameFactory, ITreeIndexFrameFactory insertLeafFrameFactory,
+            ITreeIndexFrameFactory deleteLeafFrameFactory, ILSMIndexFileManager fileManager,
+            TreeIndexFactory<BTree> diskBTreeFactory, TreeIndexFactory<BTree> bulkLoadBTreeFactory,
+            BloomFilterFactory bloomFilterFactory, double bloomFilterFalsePositiveRate,
+            IFileMapProvider diskFileMapProvider, int fieldCount, IBinaryComparatorFactory[] cmpFactories,
+            ILSMMergePolicy mergePolicy, ILSMOperationTracker opTracker, ILSMIOOperationScheduler ioScheduler,
+            ILSMIOOperationCallback ioOpCallback, boolean needKeyDupCheck) {
+        super(diskBTreeFactory.getBufferCache(), fileManager, diskFileMapProvider, bloomFilterFalsePositiveRate,
+                mergePolicy, opTracker, ioScheduler, ioOpCallback);
+        this.insertLeafFrameFactory = insertLeafFrameFactory;
+        this.deleteLeafFrameFactory = deleteLeafFrameFactory;
+        this.cmpFactories = cmpFactories;
+        componentFactory = new LSMBTreeDiskComponentFactory(diskBTreeFactory, bloomFilterFactory);
+        bulkLoadComponentFactory = new LSMBTreeDiskComponentFactory(bulkLoadBTreeFactory, bloomFilterFactory);
+        this.needKeyDupCheck = needKeyDupCheck;
+    }
+
     @Override
     public synchronized void create() throws HyracksDataException {
         if (isActivated) {
@@ -489,8 +507,9 @@
         return mergedComponent;
     }
 
-    private LSMBTreeDiskComponent createDiskComponent(LSMBTreeDiskComponentFactory factory, FileReference btreeFileRef,
-            FileReference bloomFilterFileRef, boolean createComponent) throws HyracksDataException, IndexException {
+    protected LSMBTreeDiskComponent createDiskComponent(LSMBTreeDiskComponentFactory factory,
+            FileReference btreeFileRef, FileReference bloomFilterFileRef, boolean createComponent)
+            throws HyracksDataException, IndexException {
         // Create new BTree instance.
         LSMBTreeDiskComponent component = (LSMBTreeDiskComponent) factory
                 .createLSMComponentInstance(new LSMComponentFileReferences(btreeFileRef, null, bloomFilterFileRef));
@@ -514,7 +533,7 @@
         }
     }
 
-    private ILSMComponent createBulkLoadTarget() throws HyracksDataException, IndexException {
+    protected ILSMComponent createBulkLoadTarget() throws HyracksDataException, IndexException {
         LSMComponentFileReferences componentFileRefs = fileManager.getRelFlushFileReference();
         return createDiskComponent(bulkLoadComponentFactory, componentFileRefs.getInsertIndexFileReference(),
                 componentFileRefs.getBloomFilterFileReference(), true);
diff --git a/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/impls/LSMBTreeFileManager.java b/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/impls/LSMBTreeFileManager.java
index 7e8fbb3..2f39e70 100644
--- a/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/impls/LSMBTreeFileManager.java
+++ b/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/impls/LSMBTreeFileManager.java
@@ -17,6 +17,9 @@
 
 import java.io.File;
 import java.io.FilenameFilter;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.nio.file.Paths;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.Date;
@@ -78,16 +81,19 @@
         ArrayList<ComparableFileName> allBTreeFiles = new ArrayList<ComparableFileName>();
         ArrayList<ComparableFileName> allBloomFilterFiles = new ArrayList<ComparableFileName>();
 
+        // create transaction filter <to hide transaction files>
+        FilenameFilter transactionFilter = getTransactionFileFilter(false);
+
         // Gather files
 
         // List of valid BTree files.
-        cleanupAndGetValidFilesInternal(btreeFilter, btreeFactory, allBTreeFiles);
+        cleanupAndGetValidFilesInternal(getCompoundFilter(transactionFilter, btreeFilter), btreeFactory, allBTreeFiles);
         HashSet<String> btreeFilesSet = new HashSet<String>();
         for (ComparableFileName cmpFileName : allBTreeFiles) {
             int index = cmpFileName.fileName.lastIndexOf(SPLIT_STRING);
             btreeFilesSet.add(cmpFileName.fileName.substring(0, index));
         }
-        validateFiles(btreeFilesSet, allBloomFilterFiles, bloomFilterFilter, null);
+        validateFiles(btreeFilesSet, allBloomFilterFiles, getCompoundFilter(transactionFilter, bloomFilterFilter), null);
 
         // Sanity check.
         if (allBTreeFiles.size() != allBloomFilterFiles.size()) {
@@ -159,4 +165,59 @@
 
         return validFiles;
     }
+
+    @Override
+    public LSMComponentFileReferences getNewTransactionFileReference() throws IOException {
+        Date date = new Date();
+        String ts = formatter.format(date);
+        // Create transaction lock file
+        Files.createFile(Paths.get(baseDir + TRANSACTION_PREFIX + ts));
+
+        String baseName = baseDir + ts + SPLIT_STRING + ts;
+        // Begin timestamp and end timestamp are identical since it is a transaction
+        return new LSMComponentFileReferences(createFlushFile(baseName + SPLIT_STRING + BTREE_STRING), null,
+                createFlushFile(baseName + SPLIT_STRING + BLOOM_FILTER_STRING));
+    }
+
+    @Override
+    public LSMComponentFileReferences getTransactionFileReferenceForCommit() throws HyracksDataException {
+        FilenameFilter transactionFilter;
+        File dir = new File(baseDir);
+        String[] files = dir.list(transactionFileNameFilter);
+        if (files.length == 0) {
+            return null;
+        }
+        if (files.length != 1) {
+            throw new HyracksDataException("More than one transaction lock found:" + files.length);
+        } else {
+            transactionFilter = getTransactionFileFilter(true);
+            String txnFileName = dir.getPath() + File.separator + files[0];
+            // get the actual transaction files
+            files = dir.list(transactionFilter);
+            if (files.length < 2) {
+                throw new HyracksDataException("LSM Btree transaction has less than 2 files :" + files.length);
+            }
+            try {
+                Files.delete(Paths.get(txnFileName));
+            } catch (IOException e) {
+                throw new HyracksDataException("Failed to delete transaction lock :" + txnFileName);
+            }
+        }
+        File bTreeFile = null;
+        File bloomFilterFile = null;
+
+        for (String fileName : files) {
+            if (fileName.endsWith(BTREE_STRING)) {
+                bTreeFile = new File(dir.getPath() + File.separator + fileName);
+            } else if (fileName.endsWith(BLOOM_FILTER_STRING)) {
+                bloomFilterFile = new File(dir.getPath() + File.separator + fileName);
+            } else {
+                throw new HyracksDataException("unrecognized file found = " + fileName);
+            }
+        }
+        FileReference bTreeFileRef = new FileReference(bTreeFile);
+        FileReference bloomFilterFileRef = new FileReference(bloomFilterFile);
+
+        return new LSMComponentFileReferences(bTreeFileRef, null, bloomFilterFileRef);
+    }
 }
diff --git a/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/impls/LSMBTreeWithBuddyAbstractCursor.java b/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/impls/LSMBTreeWithBuddyAbstractCursor.java
new file mode 100644
index 0000000..6316480
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/impls/LSMBTreeWithBuddyAbstractCursor.java
@@ -0,0 +1,187 @@
+/*
+ * 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.hyracks.storage.am.lsm.btree.impls;
+
+import java.util.List;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
+import edu.uci.ics.hyracks.storage.am.btree.api.IBTreeLeafFrame;
+import edu.uci.ics.hyracks.storage.am.btree.impls.BTree;
+import edu.uci.ics.hyracks.storage.am.btree.impls.BTreeRangeSearchCursor;
+import edu.uci.ics.hyracks.storage.am.btree.impls.RangePredicate;
+import edu.uci.ics.hyracks.storage.am.common.api.ICursorInitialState;
+import edu.uci.ics.hyracks.storage.am.common.api.ISearchPredicate;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexAccessor;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexCursor;
+import edu.uci.ics.hyracks.storage.am.common.api.IndexException;
+import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallback;
+import edu.uci.ics.hyracks.storage.am.common.ophelpers.MultiComparator;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMComponent;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMHarness;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndexOperationContext;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMComponent.LSMComponentType;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.BloomFilterAwareBTreePointSearchCursor;
+import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
+import edu.uci.ics.hyracks.storage.common.buffercache.ICachedPage;
+
+public abstract class LSMBTreeWithBuddyAbstractCursor implements
+		ITreeIndexCursor {
+
+	protected boolean open;
+	protected ITreeIndexCursor[] btreeCursors;
+	protected ITreeIndexCursor[] buddyBtreeCursors;
+	protected ITreeIndexAccessor[] btreeAccessors;
+	protected ITreeIndexAccessor[] buddyBtreeAccessors;
+	protected MultiComparator btreeCmp;
+	protected MultiComparator buddyBtreeCmp;
+	protected int numberOfTrees;
+	protected RangePredicate btreeRangePredicate;
+	protected RangePredicate buddyBtreeRangePredicate;
+	protected ITupleReference frameTuple;
+	protected boolean includeMutableComponent;
+	protected ILSMHarness lsmHarness;
+	protected boolean foundNext;
+	protected final ILSMIndexOperationContext opCtx;
+
+	protected List<ILSMComponent> operationalComponents;
+
+	public LSMBTreeWithBuddyAbstractCursor(ILSMIndexOperationContext opCtx) {
+		super();
+		this.opCtx = opCtx;
+	}
+
+	public ITreeIndexCursor getCursor(int cursorIndex) {
+		return btreeCursors[cursorIndex];
+	}
+
+	@Override
+	public void open(ICursorInitialState initialState,
+			ISearchPredicate searchPred) throws IndexException,
+			HyracksDataException {
+
+		LSMBTreeWithBuddyCursorInitialState lsmInitialState = (LSMBTreeWithBuddyCursorInitialState) initialState;
+		btreeCmp = lsmInitialState.getBTreeCmp();
+		buddyBtreeCmp = lsmInitialState.getBuddyBTreeCmp();
+
+		operationalComponents = lsmInitialState.getOperationalComponents();
+		lsmHarness = lsmInitialState.getLSMHarness();
+		numberOfTrees = operationalComponents.size();
+
+		btreeCursors = new ITreeIndexCursor[numberOfTrees];
+		buddyBtreeCursors = new ITreeIndexCursor[numberOfTrees];
+		btreeAccessors = new ITreeIndexAccessor[numberOfTrees];
+		buddyBtreeAccessors = new ITreeIndexAccessor[numberOfTrees];
+
+		includeMutableComponent = false;
+
+		for (int i = 0; i < numberOfTrees; i++) {
+			ILSMComponent component = operationalComponents.get(i);
+			BTree btree;
+			BTree buddyBtree;
+			if (component.getType() == LSMComponentType.MEMORY) {
+				// This is not needed at the moment but is implemented anyway
+				includeMutableComponent = true;
+				// No need for a bloom filter for the in-memory BTree.
+				buddyBtreeCursors[i] = new BTreeRangeSearchCursor(
+						(IBTreeLeafFrame) lsmInitialState
+								.getBuddyBTreeLeafFrameFactory().createFrame(),
+						false);
+				btree = ((LSMBTreeWithBuddyMemoryComponent) component)
+						.getBTree();
+				buddyBtree = ((LSMBTreeWithBuddyMemoryComponent) component)
+						.getBuddyBTree();
+			} else {
+				buddyBtreeCursors[i] = new BloomFilterAwareBTreePointSearchCursor(
+						(IBTreeLeafFrame) lsmInitialState
+								.getBuddyBTreeLeafFrameFactory().createFrame(),
+						false,
+						((LSMBTreeWithBuddyDiskComponent) operationalComponents
+								.get(i)).getBloomFilter());
+				btree = ((LSMBTreeWithBuddyDiskComponent) component).getBTree();
+				buddyBtree = (BTree) ((LSMBTreeWithBuddyDiskComponent) component)
+						.getBuddyBTree();
+			}
+			IBTreeLeafFrame leafFrame = (IBTreeLeafFrame) lsmInitialState
+					.getBTreeLeafFrameFactory().createFrame();
+			btreeCursors[i] = new BTreeRangeSearchCursor(leafFrame, false);
+			btreeAccessors[i] = btree.createAccessor(
+					NoOpOperationCallback.INSTANCE,
+					NoOpOperationCallback.INSTANCE);
+			buddyBtreeAccessors[i] = buddyBtree.createAccessor(
+					NoOpOperationCallback.INSTANCE,
+					NoOpOperationCallback.INSTANCE);
+		}
+		btreeRangePredicate = (RangePredicate) searchPred;
+		buddyBtreeRangePredicate = new RangePredicate(null, null, true, true,
+				buddyBtreeCmp, buddyBtreeCmp);
+
+		open = true;
+	}
+
+	@Override
+	public void close() throws HyracksDataException {
+		if (!open) {
+			return;
+		}
+		try {
+			if (btreeCursors != null && buddyBtreeCursors != null) {
+				for (int i = 0; i < numberOfTrees; i++) {
+					btreeCursors[i].close();
+					buddyBtreeCursors[i].close();
+				}
+			}
+			btreeCursors = null;
+			buddyBtreeCursors = null;
+		} finally {
+			lsmHarness.endSearch(opCtx);
+		}
+		foundNext = false;
+		open = false;
+	}
+
+	@Override
+	public ITupleReference getTuple() {
+		return frameTuple;
+	}
+
+	@Override
+	public ICachedPage getPage() {
+		// Do nothing
+		return null;
+	}
+
+	@Override
+	public void setBufferCache(IBufferCache bufferCache) {
+		// Do nothing
+	}
+
+	@Override
+	public void setFileId(int fileId) {
+		// Do nothing
+	}
+
+	@Override
+	public boolean exclusiveLatchNodes() {
+		return false;
+	}
+
+	@Override
+	public void markCurrentTupleAsUpdated() throws HyracksDataException {
+		throw new HyracksDataException(
+				"Updating tuples is not supported with this cursor.");
+	}
+
+}
diff --git a/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/impls/LSMBTreeWithBuddyCursorInitialState.java b/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/impls/LSMBTreeWithBuddyCursorInitialState.java
new file mode 100644
index 0000000..402dddb
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/impls/LSMBTreeWithBuddyCursorInitialState.java
@@ -0,0 +1,113 @@
+/*
+ * 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.hyracks.storage.am.lsm.btree.impls;
+
+import java.util.List;
+
+import edu.uci.ics.hyracks.storage.am.common.api.ICursorInitialState;
+import edu.uci.ics.hyracks.storage.am.common.api.ISearchOperationCallback;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
+import edu.uci.ics.hyracks.storage.am.common.ophelpers.MultiComparator;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMComponent;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMHarness;
+import edu.uci.ics.hyracks.storage.common.buffercache.ICachedPage;
+
+public class LSMBTreeWithBuddyCursorInitialState implements ICursorInitialState {
+	private final ITreeIndexFrameFactory btreeInteriorFrameFactory;
+	private final ITreeIndexFrameFactory btreeLeafFrameFactory;
+	private final ITreeIndexFrameFactory buddyBtreeLeafFrameFactory;
+	private MultiComparator btreeCmp;
+	private MultiComparator buddyBtreeCmp;
+	private final ILSMHarness lsmHarness;
+
+	private ISearchOperationCallback searchCallback;
+	private final List<ILSMComponent> operationalComponents;
+
+	public LSMBTreeWithBuddyCursorInitialState(
+			ITreeIndexFrameFactory btreeInteriorFrameFactory,
+			ITreeIndexFrameFactory btreeLeafFrameFactory,
+			ITreeIndexFrameFactory buddyBtreeLeafFrameFactory,
+			ILSMHarness lsmHarness,
+			MultiComparator btreeCmp, MultiComparator buddyBtreeCmp,
+			ISearchOperationCallback searchCallback,
+			List<ILSMComponent> operationalComponents) {
+		this.btreeLeafFrameFactory = btreeLeafFrameFactory;
+		this.btreeInteriorFrameFactory = btreeInteriorFrameFactory;
+		this.buddyBtreeLeafFrameFactory = buddyBtreeLeafFrameFactory;
+		this.btreeCmp = btreeCmp;
+		this.buddyBtreeCmp = buddyBtreeCmp;
+		this.lsmHarness = lsmHarness;
+		this.searchCallback = searchCallback;
+		this.operationalComponents = operationalComponents;
+	}
+
+	public ITreeIndexFrameFactory getBTreeInteriorFrameFactory() {
+		return btreeInteriorFrameFactory;
+	}
+
+	public ITreeIndexFrameFactory getBTreeLeafFrameFactory() {
+		return btreeLeafFrameFactory;
+	}
+
+	public ITreeIndexFrameFactory getBuddyBTreeLeafFrameFactory() {
+		return buddyBtreeLeafFrameFactory;
+	}
+
+	public MultiComparator getBTreeCmp() {
+		return btreeCmp;
+	}
+
+	public MultiComparator getBuddyBTreeCmp() {
+		return buddyBtreeCmp;
+	}
+
+	public List<ILSMComponent> getOperationalComponents() {
+		return operationalComponents;
+	}
+
+	public ILSMHarness getLSMHarness() {
+		return lsmHarness;
+	}
+
+	@Override
+	public ICachedPage getPage() {
+		return null;
+	}
+
+	@Override
+	public void setPage(ICachedPage page) {
+	}
+
+	@Override
+	public ISearchOperationCallback getSearchOperationCallback() {
+		return searchCallback;
+	}
+
+	@Override
+	public void setSearchOperationCallback(
+			ISearchOperationCallback searchCallback) {
+		this.searchCallback = searchCallback;
+	}
+
+	@Override
+	public MultiComparator getOriginalKeyComparator() {
+		return btreeCmp;
+	}
+
+	@Override
+	public void setOriginialKeyComparator(MultiComparator originalCmp) {
+		this.btreeCmp = originalCmp;
+	}
+}
diff --git a/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/impls/LSMBTreeWithBuddyDiskComponent.java b/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/impls/LSMBTreeWithBuddyDiskComponent.java
new file mode 100644
index 0000000..85063de
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/impls/LSMBTreeWithBuddyDiskComponent.java
@@ -0,0 +1,65 @@
+/*
+ * 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.hyracks.storage.am.lsm.btree.impls;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.storage.am.bloomfilter.impls.BloomFilter;
+import edu.uci.ics.hyracks.storage.am.btree.impls.BTree;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.AbstractDiskLSMComponent;
+
+public class LSMBTreeWithBuddyDiskComponent extends AbstractDiskLSMComponent {
+
+	private final BTree btree;
+	private final BTree buddyBtree;
+	private final BloomFilter bloomFilter;
+
+	public LSMBTreeWithBuddyDiskComponent(BTree btree, BTree buddyBtree,
+			BloomFilter bloomFilter) {
+		this.btree = btree;
+		this.buddyBtree = buddyBtree;
+		this.bloomFilter = bloomFilter;
+	}
+
+	@Override
+	protected void destroy() throws HyracksDataException {
+		btree.deactivate();
+		btree.destroy();
+		buddyBtree.deactivate();
+		buddyBtree.destroy();
+		bloomFilter.deactivate();
+		bloomFilter.destroy();
+	}
+
+	public BTree getBTree() {
+		return btree;
+	}
+
+	public BTree getBuddyBTree() {
+		return buddyBtree;
+	}
+
+	public BloomFilter getBloomFilter() {
+		return bloomFilter;
+	}
+
+	@Override
+	public long getComponentSize() {
+		long size = btree.getFileReference().getFile().length();
+		size += buddyBtree.getFileReference().getFile().length();
+		size += bloomFilter.getFileReference().getFile().length();
+		return size;
+	}
+
+}
diff --git a/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/impls/LSMBTreeWithBuddyDiskComponentFactory.java b/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/impls/LSMBTreeWithBuddyDiskComponentFactory.java
new file mode 100644
index 0000000..aa6256a
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/impls/LSMBTreeWithBuddyDiskComponentFactory.java
@@ -0,0 +1,61 @@
+/*
+ * 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.hyracks.storage.am.lsm.btree.impls;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.storage.am.bloomfilter.impls.BloomFilterFactory;
+import edu.uci.ics.hyracks.storage.am.btree.impls.BTree;
+import edu.uci.ics.hyracks.storage.am.common.api.IndexException;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMComponent;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMComponentFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.LSMComponentFileReferences;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.TreeIndexFactory;
+import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
+
+public class LSMBTreeWithBuddyDiskComponentFactory implements
+		ILSMComponentFactory {
+
+	private final TreeIndexFactory<BTree> btreeFactory;
+	private final TreeIndexFactory<BTree> buddyBtreeFactory;
+	private final BloomFilterFactory bloomFilterFactory;
+
+	public LSMBTreeWithBuddyDiskComponentFactory(
+			TreeIndexFactory<BTree> btreeFactory,
+			TreeIndexFactory<BTree> buddyBtreeFactory,
+			BloomFilterFactory bloomFilterFactory) {
+		this.btreeFactory = btreeFactory;
+		this.buddyBtreeFactory = buddyBtreeFactory;
+		this.bloomFilterFactory = bloomFilterFactory;
+	}
+
+	@Override
+	public ILSMComponent createLSMComponentInstance(
+			LSMComponentFileReferences cfr) throws IndexException,
+			HyracksDataException {
+		return new LSMBTreeWithBuddyDiskComponent(
+				btreeFactory.createIndexInstance(cfr
+						.getInsertIndexFileReference()),
+				buddyBtreeFactory.createIndexInstance(cfr
+						.getDeleteIndexFileReference()),
+				bloomFilterFactory.createBloomFiltertInstance(cfr
+						.getBloomFilterFileReference()));
+	}
+
+	@Override
+	public IBufferCache getBufferCache() {
+		return btreeFactory.getBufferCache();
+	}
+
+}
diff --git a/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/impls/LSMBTreeWithBuddyFileManager.java b/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/impls/LSMBTreeWithBuddyFileManager.java
new file mode 100644
index 0000000..51ee0fb
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/impls/LSMBTreeWithBuddyFileManager.java
@@ -0,0 +1,257 @@
+/*
+ * 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.hyracks.storage.am.lsm.btree.impls;
+
+import java.io.File;
+import java.io.FilenameFilter;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Date;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.io.FileReference;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
+import edu.uci.ics.hyracks.storage.am.common.api.IndexException;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.AbstractLSMIndexFileManager;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.LSMComponentFileReferences;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.TreeIndexFactory;
+import edu.uci.ics.hyracks.storage.common.file.IFileMapProvider;
+
+public class LSMBTreeWithBuddyFileManager extends AbstractLSMIndexFileManager {
+    private static final String BUDDY_BTREE_STRING = "buddy";
+    private static final String BTREE_STRING = "b";
+
+    private final TreeIndexFactory<? extends ITreeIndex> btreeFactory;
+    private final TreeIndexFactory<? extends ITreeIndex> buddyBtreeFactory;
+
+    private static FilenameFilter btreeFilter = new FilenameFilter() {
+        public boolean accept(File dir, String name) {
+            return !name.startsWith(".") && name.endsWith(BTREE_STRING);
+        }
+    };
+
+    private static FilenameFilter buddyBtreeFilter = new FilenameFilter() {
+        public boolean accept(File dir, String name) {
+            return !name.startsWith(".") && name.endsWith(BUDDY_BTREE_STRING);
+        }
+    };
+
+    public LSMBTreeWithBuddyFileManager(IFileMapProvider fileMapProvider, FileReference file,
+            TreeIndexFactory<? extends ITreeIndex> btreeFactory,
+            TreeIndexFactory<? extends ITreeIndex> buddyBtreeFactory) {
+        super(fileMapProvider, file, null);
+        this.buddyBtreeFactory = buddyBtreeFactory;
+        this.btreeFactory = btreeFactory;
+    }
+
+    @Override
+    public LSMComponentFileReferences getRelFlushFileReference() {
+        Date date = new Date();
+        String ts = formatter.format(date);
+        String baseName = baseDir + ts + SPLIT_STRING + ts;
+        // Begin timestamp and end timestamp are identical since it is a flush
+        return new LSMComponentFileReferences(createFlushFile(baseName + SPLIT_STRING + BTREE_STRING),
+                createFlushFile(baseName + SPLIT_STRING + BUDDY_BTREE_STRING), createFlushFile(baseName + SPLIT_STRING
+                        + BLOOM_FILTER_STRING));
+    }
+
+    @Override
+    public LSMComponentFileReferences getRelMergeFileReference(String firstFileName, String lastFileName)
+            throws HyracksDataException {
+        String[] firstTimestampRange = firstFileName.split(SPLIT_STRING);
+        String[] lastTimestampRange = lastFileName.split(SPLIT_STRING);
+
+        String baseName = baseDir + firstTimestampRange[0] + SPLIT_STRING + lastTimestampRange[1];
+        // Get the range of timestamps by taking the earliest and the latest
+        // timestamps
+        return new LSMComponentFileReferences(createMergeFile(baseName + SPLIT_STRING + BTREE_STRING),
+                createMergeFile(baseName + SPLIT_STRING + BUDDY_BTREE_STRING), createMergeFile(baseName + SPLIT_STRING
+                        + BLOOM_FILTER_STRING));
+    }
+
+    @Override
+    public List<LSMComponentFileReferences> cleanupAndGetValidFiles() throws HyracksDataException, IndexException {
+        List<LSMComponentFileReferences> validFiles = new ArrayList<LSMComponentFileReferences>();
+        ArrayList<ComparableFileName> allBTreeFiles = new ArrayList<ComparableFileName>();
+        ArrayList<ComparableFileName> allBuddyBTreeFiles = new ArrayList<ComparableFileName>();
+        ArrayList<ComparableFileName> allBloomFilterFiles = new ArrayList<ComparableFileName>();
+
+        // Create transaction file filter
+        FilenameFilter transactionFilefilter = getTransactionFileFilter(false);
+
+        // Gather files.
+        cleanupAndGetValidFilesInternal(getCompoundFilter(btreeFilter, transactionFilefilter), btreeFactory,
+                allBTreeFiles);
+        HashSet<String> btreeFilesSet = new HashSet<String>();
+        for (ComparableFileName cmpFileName : allBTreeFiles) {
+            int index = cmpFileName.fileName.lastIndexOf(SPLIT_STRING);
+            btreeFilesSet.add(cmpFileName.fileName.substring(0, index));
+        }
+        validateFiles(btreeFilesSet, allBuddyBTreeFiles, getCompoundFilter(buddyBtreeFilter, transactionFilefilter),
+                buddyBtreeFactory);
+        validateFiles(btreeFilesSet, allBloomFilterFiles, getCompoundFilter(bloomFilterFilter, transactionFilefilter),
+                null);
+
+        // Sanity check.
+        if (allBTreeFiles.size() != allBuddyBTreeFiles.size() || allBTreeFiles.size() != allBloomFilterFiles.size()) {
+            throw new HyracksDataException(
+                    "Unequal number of valid BTree, Buddy BTree, and Bloom Filter files found. Aborting cleanup.");
+        }
+
+        // Trivial cases.
+        if (allBTreeFiles.isEmpty() || allBuddyBTreeFiles.isEmpty() || allBloomFilterFiles.isEmpty()) {
+            return validFiles;
+        }
+
+        if (allBTreeFiles.size() == 1 && allBuddyBTreeFiles.size() == 1 && allBloomFilterFiles.size() == 1) {
+            validFiles.add(new LSMComponentFileReferences(allBTreeFiles.get(0).fileRef,
+                    allBuddyBTreeFiles.get(0).fileRef, allBloomFilterFiles.get(0).fileRef));
+            return validFiles;
+        }
+
+        // Sorts files names from earliest to latest timestamp.
+        Collections.sort(allBTreeFiles);
+        Collections.sort(allBuddyBTreeFiles);
+        Collections.sort(allBloomFilterFiles);
+
+        List<ComparableFileName> validComparableBTreeFiles = new ArrayList<ComparableFileName>();
+        ComparableFileName lastBTree = allBTreeFiles.get(0);
+        validComparableBTreeFiles.add(lastBTree);
+
+        List<ComparableFileName> validComparableBuddyBTreeFiles = new ArrayList<ComparableFileName>();
+        ComparableFileName lastBuddyBTree = allBuddyBTreeFiles.get(0);
+        validComparableBuddyBTreeFiles.add(lastBuddyBTree);
+
+        List<ComparableFileName> validComparableBloomFilterFiles = new ArrayList<ComparableFileName>();
+        ComparableFileName lastBloomFilter = allBloomFilterFiles.get(0);
+        validComparableBloomFilterFiles.add(lastBloomFilter);
+
+        for (int i = 1; i < allBTreeFiles.size(); i++) {
+            ComparableFileName currentBTree = allBTreeFiles.get(i);
+            ComparableFileName currentBuddyBTree = allBuddyBTreeFiles.get(i);
+            ComparableFileName currentBloomFilter = allBloomFilterFiles.get(i);
+            // Current start timestamp is greater than last stop timestamp.
+            if (currentBTree.interval[0].compareTo(lastBTree.interval[1]) > 0
+                    && currentBuddyBTree.interval[0].compareTo(lastBuddyBTree.interval[1]) > 0
+                    && currentBloomFilter.interval[0].compareTo(lastBloomFilter.interval[1]) > 0) {
+                validComparableBTreeFiles.add(currentBTree);
+                validComparableBuddyBTreeFiles.add(currentBuddyBTree);
+                validComparableBloomFilterFiles.add(currentBloomFilter);
+                lastBTree = currentBTree;
+                lastBuddyBTree = currentBuddyBTree;
+                lastBloomFilter = currentBloomFilter;
+            } else if (currentBTree.interval[0].compareTo(lastBTree.interval[0]) >= 0
+                    && currentBTree.interval[1].compareTo(lastBTree.interval[1]) <= 0
+                    && currentBuddyBTree.interval[0].compareTo(lastBuddyBTree.interval[0]) >= 0
+                    && currentBuddyBTree.interval[1].compareTo(lastBuddyBTree.interval[1]) <= 0
+                    && currentBloomFilter.interval[0].compareTo(lastBloomFilter.interval[0]) >= 0
+                    && currentBloomFilter.interval[1].compareTo(lastBloomFilter.interval[1]) <= 0) {
+                // Invalid files are completely contained in last interval.
+                File invalidBTreeFile = new File(currentBTree.fullPath);
+                invalidBTreeFile.delete();
+                File invalidBuddyBTreeFile = new File(currentBuddyBTree.fullPath);
+                invalidBuddyBTreeFile.delete();
+                File invalidBloomFilterFile = new File(currentBloomFilter.fullPath);
+                invalidBloomFilterFile.delete();
+            } else {
+                // This scenario should not be possible.
+                throw new HyracksDataException("Found LSM files with overlapping but not contained timetamp intervals.");
+            }
+        }
+
+        // Sort valid files in reverse lexicographical order, such that newer
+        // files come first.
+        Collections.sort(validComparableBTreeFiles, recencyCmp);
+        Collections.sort(validComparableBuddyBTreeFiles, recencyCmp);
+        Collections.sort(validComparableBloomFilterFiles, recencyCmp);
+
+        Iterator<ComparableFileName> btreeFileIter = validComparableBTreeFiles.iterator();
+        Iterator<ComparableFileName> buddyBtreeFileIter = validComparableBuddyBTreeFiles.iterator();
+        Iterator<ComparableFileName> bloomFilterFileIter = validComparableBloomFilterFiles.iterator();
+        while (btreeFileIter.hasNext() && buddyBtreeFileIter.hasNext()) {
+            ComparableFileName cmpBTreeFileName = btreeFileIter.next();
+            ComparableFileName cmpBuddyBTreeFileName = buddyBtreeFileIter.next();
+            ComparableFileName cmpBloomFilterFileName = bloomFilterFileIter.next();
+            validFiles.add(new LSMComponentFileReferences(cmpBTreeFileName.fileRef, cmpBuddyBTreeFileName.fileRef,
+                    cmpBloomFilterFileName.fileRef));
+        }
+
+        return validFiles;
+    }
+
+    @Override
+    public LSMComponentFileReferences getNewTransactionFileReference() throws IOException {
+        Date date = new Date();
+        String ts = formatter.format(date);
+        // Create transaction lock file
+        Files.createFile(Paths.get(baseDir + TRANSACTION_PREFIX + ts));
+
+        String baseName = baseDir + ts + SPLIT_STRING + ts;
+        return new LSMComponentFileReferences(createFlushFile(baseName + SPLIT_STRING + BTREE_STRING),
+                createFlushFile(baseName + SPLIT_STRING + BUDDY_BTREE_STRING), createFlushFile(baseName + SPLIT_STRING
+                        + BLOOM_FILTER_STRING));
+    }
+
+    @Override
+    public LSMComponentFileReferences getTransactionFileReferenceForCommit() throws HyracksDataException {
+        FilenameFilter transactionFilter;
+        File dir = new File(baseDir);
+        String[] files = dir.list(transactionFileNameFilter);
+        if (files.length == 0)
+            return null;
+        if (files.length != 1) {
+            throw new HyracksDataException("More than one transaction lock found:" + files.length);
+        } else {
+            transactionFilter = getTransactionFileFilter(true);
+            String txnFileName = dir.getPath() + File.separator + files[0];
+            // get the actual transaction files
+            files = dir.list(transactionFilter);
+            if (files.length < 3) {
+                throw new HyracksDataException("LSM Btree with buddy transaction has less than 3 files :"
+                        + files.length);
+            }
+            try {
+                Files.delete(Paths.get(txnFileName));
+            } catch (IOException e) {
+                throw new HyracksDataException("Failed to delete transaction lock :" + txnFileName);
+            }
+        }
+        File bTreeFile = null;
+        File buddyBTreeFile = null;
+        File bloomFilterFile = null;
+        for (String fileName : files) {
+            if (fileName.endsWith(BTREE_STRING)) {
+                bTreeFile = new File(dir.getPath() + File.separator + fileName);
+            } else if (fileName.endsWith(BUDDY_BTREE_STRING)) {
+                buddyBTreeFile = new File(dir.getPath() + File.separator + fileName);
+            } else if (fileName.endsWith(BLOOM_FILTER_STRING)) {
+                bloomFilterFile = new File(dir.getPath() + File.separator + fileName);
+            } else {
+                throw new HyracksDataException("unrecognized file found = " + fileName);
+            }
+        }
+        FileReference bTreeFileRef = new FileReference(bTreeFile);
+        FileReference buddyBTreeFileRef = new FileReference(buddyBTreeFile);
+        FileReference bloomFilterFileRef = new FileReference(bloomFilterFile);
+        return new LSMComponentFileReferences(bTreeFileRef, buddyBTreeFileRef, bloomFilterFileRef);
+    }
+
+}
diff --git a/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/impls/LSMBTreeWithBuddyMemoryComponent.java b/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/impls/LSMBTreeWithBuddyMemoryComponent.java
new file mode 100644
index 0000000..34509a3
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/impls/LSMBTreeWithBuddyMemoryComponent.java
@@ -0,0 +1,59 @@
+/*
+ * 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.hyracks.storage.am.lsm.btree.impls;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.storage.am.btree.impls.BTree;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.IVirtualBufferCache;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.AbstractMemoryLSMComponent;
+
+/*
+ * This class is also not needed at the moment but is implemented anyway
+ */
+public class LSMBTreeWithBuddyMemoryComponent extends
+		AbstractMemoryLSMComponent {
+
+	private final BTree btree;
+	private final BTree buddyBtree;
+
+	public LSMBTreeWithBuddyMemoryComponent(BTree btree, BTree buddyBtree,
+			IVirtualBufferCache vbc, boolean isActive) {
+		super(vbc, isActive);
+		this.btree = btree;
+		this.buddyBtree = buddyBtree;
+	}
+
+	public BTree getBTree() {
+		return btree;
+	}
+
+	public BTree getBuddyBTree() {
+		return buddyBtree;
+	}
+
+	@Override
+	protected void reset() throws HyracksDataException {
+		super.reset();
+		btree.deactivate();
+		btree.destroy();
+		btree.create();
+		btree.activate();
+		buddyBtree.deactivate();
+		buddyBtree.destroy();
+		buddyBtree.create();
+		buddyBtree.activate();
+	}
+
+}
diff --git a/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/impls/LSMBTreeWithBuddyMergeOperation.java b/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/impls/LSMBTreeWithBuddyMergeOperation.java
new file mode 100644
index 0000000..4bae2c8
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/impls/LSMBTreeWithBuddyMergeOperation.java
@@ -0,0 +1,132 @@
+/*
+ * 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.hyracks.storage.am.lsm.btree.impls;
+
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.io.FileReference;
+import edu.uci.ics.hyracks.api.io.IODeviceHandle;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexCursor;
+import edu.uci.ics.hyracks.storage.am.common.api.IndexException;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMComponent;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndexAccessorInternal;
+
+public class LSMBTreeWithBuddyMergeOperation implements ILSMIOOperation {
+
+	private final ILSMIndexAccessorInternal accessor;
+	private final List<ILSMComponent> mergingComponents;
+	private final ITreeIndexCursor cursor;
+	private final FileReference btreeMergeTarget;
+	private final FileReference buddyBtreeMergeTarget;
+	private final FileReference bloomFilterMergeTarget;
+	private final ILSMIOOperationCallback callback;
+	private final String indexIdentifier;
+	private final boolean keepDeletedTuples;
+
+	public LSMBTreeWithBuddyMergeOperation(ILSMIndexAccessorInternal accessor,
+			List<ILSMComponent> mergingComponents, ITreeIndexCursor cursor,
+			FileReference btreeMergeTarget,
+			FileReference buddyBtreeMergeTarget,
+			FileReference bloomFilterMergeTarget,
+			ILSMIOOperationCallback callback, String indexIdentifier, boolean keepDeletedTuples) {
+		this.accessor = accessor;
+		this.mergingComponents = mergingComponents;
+		this.cursor = cursor;
+		this.btreeMergeTarget = btreeMergeTarget;
+		this.buddyBtreeMergeTarget = buddyBtreeMergeTarget;
+		this.bloomFilterMergeTarget = bloomFilterMergeTarget;
+		this.callback = callback;
+		this.indexIdentifier = indexIdentifier;
+		this.keepDeletedTuples = keepDeletedTuples;
+	}
+
+	@Override
+	public Set<IODeviceHandle> getReadDevices() {
+		Set<IODeviceHandle> devs = new HashSet<IODeviceHandle>();
+		for (ILSMComponent o : mergingComponents) {
+			LSMBTreeWithBuddyDiskComponent component = (LSMBTreeWithBuddyDiskComponent) o;
+			devs.add(component.getBTree().getFileReference().getDeviceHandle());
+
+			devs.add(component.getBuddyBTree().getFileReference()
+					.getDeviceHandle());
+			devs.add(component.getBloomFilter().getFileReference()
+					.getDeviceHandle());
+
+		}
+		return devs;
+	}
+
+	@Override
+	public Set<IODeviceHandle> getWriteDevices() {
+		Set<IODeviceHandle> devs = new HashSet<IODeviceHandle>();
+		devs.add(btreeMergeTarget.getDeviceHandle());
+
+		devs.add(buddyBtreeMergeTarget.getDeviceHandle());
+		devs.add(bloomFilterMergeTarget.getDeviceHandle());
+
+		return devs;
+	}
+
+	@Override
+	public Boolean call() throws HyracksDataException, IndexException {
+		accessor.merge(this);
+		return true;
+	}
+
+	@Override
+	public ILSMIOOperationCallback getCallback() {
+		return callback;
+	}
+
+	@Override
+	public String getIndexUniqueIdentifier() {
+		return indexIdentifier;
+	}
+
+	@Override
+	public LSMIOOpertionType getIOOpertionType() {
+		return LSMIOOpertionType.MERGE;
+	}
+
+	public FileReference getBTreeMergeTarget() {
+		return btreeMergeTarget;
+	}
+
+	public FileReference getBuddyBTreeMergeTarget() {
+		return buddyBtreeMergeTarget;
+	}
+
+	public FileReference getBloomFilterMergeTarget() {
+		return bloomFilterMergeTarget;
+	}
+
+	public ITreeIndexCursor getCursor() {
+		return cursor;
+	}
+
+	public List<ILSMComponent> getMergingComponents() {
+		return mergingComponents;
+	}
+
+	public boolean isKeepDeletedTuples() {
+		return keepDeletedTuples;
+	}
+
+}
diff --git a/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/impls/LSMBTreeWithBuddySearchCursor.java b/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/impls/LSMBTreeWithBuddySearchCursor.java
new file mode 100644
index 0000000..f32bc52
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/impls/LSMBTreeWithBuddySearchCursor.java
@@ -0,0 +1,120 @@
+/*
+ * 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.hyracks.storage.am.lsm.btree.impls;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
+import edu.uci.ics.hyracks.storage.am.common.api.ICursorInitialState;
+import edu.uci.ics.hyracks.storage.am.common.api.ISearchPredicate;
+import edu.uci.ics.hyracks.storage.am.common.api.IndexException;
+import edu.uci.ics.hyracks.storage.am.common.tuples.PermutingTupleReference;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndexOperationContext;
+
+public class LSMBTreeWithBuddySearchCursor extends LSMBTreeWithBuddyAbstractCursor{
+	private int currentCursor;
+    private PermutingTupleReference buddyBTreeTuple;
+    
+    public LSMBTreeWithBuddySearchCursor(ILSMIndexOperationContext opCtx, int[] buddyBTreeFields) {
+        super(opCtx);
+        currentCursor = 0;
+        this.buddyBTreeTuple = new PermutingTupleReference(buddyBTreeFields);
+    }
+    
+    @Override
+    public void close() throws HyracksDataException {
+        super.close();
+        currentCursor = 0;
+    }
+
+    @Override
+    public void reset() throws HyracksDataException {
+        if (!open) {
+            return;
+        }
+
+        currentCursor = 0;
+        foundNext = false;
+        try {
+            for (int i = 0; i < numberOfTrees; i++) {
+                btreeCursors[i].close();
+                buddyBtreeCursors[i].close();
+            }
+            btreeCursors = null;
+            buddyBtreeCursors = null;
+        } finally {
+            lsmHarness.endSearch(opCtx);
+        }
+    }
+    
+    private void searchNextCursor() throws HyracksDataException {
+        if (currentCursor < numberOfTrees) {
+            try {
+            	btreeCursors[currentCursor].reset();
+                btreeAccessors[currentCursor].search(btreeCursors[currentCursor], btreeRangePredicate);
+            } catch (IndexException e) {
+                throw new HyracksDataException(e);
+            }
+        }
+    }
+    
+    @Override
+    public boolean hasNext() throws HyracksDataException, IndexException {
+        if (foundNext) {
+            return true;
+        }
+        while (currentCursor < numberOfTrees) {
+            while (btreeCursors[currentCursor].hasNext()) {
+                btreeCursors[currentCursor].next();
+                ITupleReference currentTuple = btreeCursors[currentCursor].getTuple();
+                buddyBTreeTuple.reset(btreeCursors[currentCursor].getTuple());
+                boolean killerTupleFound = false;
+                for (int i = 0; i < currentCursor; i++) {
+                    buddyBtreeCursors[i].reset();
+                    buddyBtreeRangePredicate.setHighKey(buddyBTreeTuple, true);
+                    buddyBtreeRangePredicate.setLowKey(buddyBTreeTuple, true);
+                    buddyBtreeAccessors[i].search(buddyBtreeCursors[i], buddyBtreeRangePredicate);
+                    try {
+                        if (buddyBtreeCursors[i].hasNext()) {
+                            killerTupleFound = true;
+                            break;
+                        }
+                    } finally {
+                        buddyBtreeCursors[i].close();
+                    }
+                }
+                if (!killerTupleFound) {
+                    frameTuple = currentTuple;
+                    foundNext = true;
+                    return true;
+                }
+            }
+            btreeCursors[currentCursor].close();
+            currentCursor++;
+            searchNextCursor();
+        }
+        return false;
+    }
+    
+    @Override
+    public void next() throws HyracksDataException {
+        foundNext = false;
+    }
+
+    @Override
+    public void open(ICursorInitialState initialState, ISearchPredicate searchPred) throws HyracksDataException, IndexException {
+        super.open(initialState, searchPred);
+        searchNextCursor();
+    }
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/impls/LSMBTreeWithBuddySortedCursor.java b/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/impls/LSMBTreeWithBuddySortedCursor.java
new file mode 100644
index 0000000..54a3ae8
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/impls/LSMBTreeWithBuddySortedCursor.java
@@ -0,0 +1,165 @@
+/*
+ * 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.hyracks.storage.am.lsm.btree.impls;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.storage.am.common.api.ICursorInitialState;
+import edu.uci.ics.hyracks.storage.am.common.api.ISearchPredicate;
+import edu.uci.ics.hyracks.storage.am.common.api.IndexException;
+import edu.uci.ics.hyracks.storage.am.common.tuples.PermutingTupleReference;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndexOperationContext;
+
+public class LSMBTreeWithBuddySortedCursor extends
+		LSMBTreeWithBuddyAbstractCursor {
+	// TODO: This class can be removed and instead use a search cursor that uses
+	// a logic similar
+	// to the one in LSMRTreeWithAntiMatterTuplesSearchCursor
+	private boolean[] depletedBtreeCursors;
+	private int foundIn = -1;
+	private PermutingTupleReference buddyBtreeTuple;
+
+	public LSMBTreeWithBuddySortedCursor(ILSMIndexOperationContext opCtx,
+			int[] buddyBTreeFields) throws HyracksDataException {
+		super(opCtx);
+		this.buddyBtreeTuple = new PermutingTupleReference(buddyBTreeFields);
+		reset();
+	}
+
+	public ILSMIndexOperationContext getOpCtx() {
+		return opCtx;
+	}
+
+	@Override
+	public void reset() throws HyracksDataException {
+		depletedBtreeCursors = new boolean[numberOfTrees];
+		foundNext = false;
+		try {
+			for (int i = 0; i < numberOfTrees; i++) {
+				btreeCursors[i].reset();
+				try {
+					btreeAccessors[i].search(btreeCursors[i],
+							btreeRangePredicate);
+				} catch (IndexException e) {
+					throw new HyracksDataException(e);
+				}
+				if (btreeCursors[i].hasNext()) {
+					btreeCursors[i].next();
+				} else {
+					depletedBtreeCursors[i] = true;
+				}
+			}
+		} catch (IndexException e) {
+			e.printStackTrace();
+			throw new HyracksDataException(
+					"error while reseting the btrees of the lsm btree with buddy btree",
+					e);
+		} finally {
+			if (open) {
+				lsmHarness.endSearch(opCtx);
+			}
+		}
+	}
+
+	@Override
+	public boolean hasNext() throws HyracksDataException, IndexException {
+		while (!foundNext) {
+			frameTuple = null;
+
+			if (foundIn != -1) {
+				if (btreeCursors[foundIn].hasNext()) {
+					btreeCursors[foundIn].next();
+				} else {
+					depletedBtreeCursors[foundIn] = true;
+				}
+			}
+
+			foundIn = -1;
+			for (int i = 0; i < numberOfTrees; i++) {
+				if (depletedBtreeCursors[i])
+					continue;
+
+				if (frameTuple == null) {
+					frameTuple = btreeCursors[i].getTuple();
+					foundIn = i;
+					continue;
+				}
+
+				if (btreeCmp.compare(frameTuple, btreeCursors[i].getTuple()) > 0) {
+					frameTuple = btreeCursors[i].getTuple();
+					foundIn = i;
+				}
+			}
+
+			if (foundIn == -1)
+				return false;
+
+			boolean killed = false;
+			buddyBtreeTuple.reset(frameTuple);
+			for (int i = 0; i < foundIn; i++) {
+				try {
+					buddyBtreeCursors[i].reset();
+					buddyBtreeRangePredicate.setHighKey(buddyBtreeTuple, true);
+					btreeRangePredicate.setLowKey(buddyBtreeTuple, true);
+					btreeAccessors[i].search(btreeCursors[i],
+							btreeRangePredicate);
+				} catch (IndexException e) {
+					throw new HyracksDataException(e);
+				}
+				try {
+					if (btreeCursors[i].hasNext()) {
+						killed = true;
+						break;
+					}
+				} finally {
+					btreeCursors[i].close();
+				}
+			}
+			if (!killed) {
+				foundNext = true;
+			}
+		}
+
+		return true;
+	}
+
+	@Override
+	public void next() throws HyracksDataException {
+		foundNext = false;
+	}
+
+	@Override
+	public void open(ICursorInitialState initialState,
+			ISearchPredicate searchPred) throws HyracksDataException,
+			IndexException {
+		super.open(initialState, searchPred);
+
+		depletedBtreeCursors = new boolean[numberOfTrees];
+		foundNext = false;
+		for (int i = 0; i < numberOfTrees; i++) {
+			btreeCursors[i].reset();
+			try {
+				btreeAccessors[i].search(btreeCursors[i], btreeRangePredicate);
+			} catch (IndexException e) {
+				throw new HyracksDataException(e);
+			}
+			if (btreeCursors[i].hasNext()) {
+				btreeCursors[i].next();
+			} else {
+				depletedBtreeCursors[i] = true;
+			}
+		}
+	}
+
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/impls/LSMBuddyBTreeMergeCursor.java b/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/impls/LSMBuddyBTreeMergeCursor.java
new file mode 100644
index 0000000..c650de2
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/impls/LSMBuddyBTreeMergeCursor.java
@@ -0,0 +1,77 @@
+/*
+ * 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.hyracks.storage.am.lsm.btree.impls;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.storage.am.btree.api.IBTreeLeafFrame;
+import edu.uci.ics.hyracks.storage.am.btree.impls.BTree;
+import edu.uci.ics.hyracks.storage.am.btree.impls.BTreeRangeSearchCursor;
+import edu.uci.ics.hyracks.storage.am.btree.impls.RangePredicate;
+import edu.uci.ics.hyracks.storage.am.common.api.ICursorInitialState;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexAccessor;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexCursor;
+import edu.uci.ics.hyracks.storage.am.common.api.ISearchPredicate;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexAccessor;
+import edu.uci.ics.hyracks.storage.am.common.api.IndexException;
+import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallback;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMComponent;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndexOperationContext;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.LSMIndexSearchCursor;
+
+public class LSMBuddyBTreeMergeCursor extends LSMIndexSearchCursor {
+
+	public LSMBuddyBTreeMergeCursor(ILSMIndexOperationContext opCtx) {
+		super(opCtx, true);
+	}
+
+	@Override
+	protected boolean isDeleted(PriorityQueueElement checkElement)
+			throws HyracksDataException, IndexException {
+		return false;
+	}
+
+	@Override
+	public void open(ICursorInitialState initialState,
+			ISearchPredicate searchPred) throws IndexException,
+			HyracksDataException {
+		LSMBTreeWithBuddyCursorInitialState lsmInitialState = (LSMBTreeWithBuddyCursorInitialState) initialState;
+		cmp = lsmInitialState.getBuddyBTreeCmp();
+		operationalComponents = lsmInitialState.getOperationalComponents();
+		// We intentionally set the lsmHarness to null so that we don't call
+		// lsmHarness.endSearch() because we already do that when we merge
+		// actual index.
+		lsmHarness = null;
+		int numBTrees = operationalComponents.size();
+		rangeCursors = new IIndexCursor[numBTrees];
+
+		RangePredicate btreePredicate = new RangePredicate(null, null, true,
+				true, cmp, cmp);
+		IIndexAccessor[] btreeAccessors = new ITreeIndexAccessor[numBTrees];
+		for (int i = 0; i < numBTrees; i++) {
+			ILSMComponent component = operationalComponents.get(i);
+			IBTreeLeafFrame leafFrame = (IBTreeLeafFrame) lsmInitialState
+					.getBuddyBTreeLeafFrameFactory().createFrame();
+			rangeCursors[i] = new BTreeRangeSearchCursor(leafFrame, false);
+			BTree buddyBtree = (BTree) ((LSMBTreeWithBuddyDiskComponent) component)
+					.getBuddyBTree();
+			btreeAccessors[i] = buddyBtree.createAccessor(
+					NoOpOperationCallback.INSTANCE,
+					NoOpOperationCallback.INSTANCE);
+			btreeAccessors[i].search(rangeCursors[i], btreePredicate);
+		}
+		setPriorityQueueComparator();
+		initPriorityQueue();
+	}
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/tuples/LSMBTreeRefrencingTupleWriterFactory.java b/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/tuples/LSMBTreeRefrencingTupleWriterFactory.java
new file mode 100644
index 0000000..7582a33
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/tuples/LSMBTreeRefrencingTupleWriterFactory.java
@@ -0,0 +1,65 @@
+/*
+ * 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.hyracks.storage.am.lsm.btree.tuples;
+
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexTupleWriter;
+import edu.uci.ics.hyracks.storage.am.common.ophelpers.IndexOperation;
+import edu.uci.ics.hyracks.storage.am.common.tuples.TypeAwareTupleWriterFactory;
+
+/**
+ * This is a TupleWriterFactory that can be used to reference the last writer it created
+ * and switch between actual and antimatter writer modes
+ * @author alamouda
+ *
+ */
+public class LSMBTreeRefrencingTupleWriterFactory extends TypeAwareTupleWriterFactory{
+	private static final long serialVersionUID = 1L;
+	private final ITypeTraits[] typeTraits;
+	private final int numKeyFields;
+	private boolean isDelete;
+	private LSMBTreeTupleWriter createdTupleWriter;
+
+	public LSMBTreeRefrencingTupleWriterFactory(ITypeTraits[] typeTraits, int numKeyFields, boolean isDelete) {
+		super(typeTraits);
+		this.typeTraits = typeTraits;
+		this.numKeyFields = numKeyFields;
+		this.isDelete = isDelete;
+	}
+
+	@Override
+	public ITreeIndexTupleWriter createTupleWriter() {
+		createdTupleWriter = new LSMBTreeTupleWriter(typeTraits, numKeyFields, isDelete);
+		return createdTupleWriter;
+	}
+
+	public void setMode(IndexOperation op){
+		if(op == IndexOperation.INSERT){
+			this.isDelete = false;
+			if(createdTupleWriter != null){
+				this.createdTupleWriter.setAntimatter(false);
+			}
+		} else if(op == IndexOperation.DELETE){
+			this.isDelete = true;
+			if(createdTupleWriter != null){
+				this.createdTupleWriter.setAntimatter(true);
+			}
+		}
+	}
+
+	public LSMBTreeTupleWriter getCreatedTupleWriter() {
+		return createdTupleWriter;
+	}
+}
diff --git a/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/tuples/LSMBTreeTupleWriter.java b/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/tuples/LSMBTreeTupleWriter.java
index 9426282..2206b1e 100644
--- a/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/tuples/LSMBTreeTupleWriter.java
+++ b/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/tuples/LSMBTreeTupleWriter.java
@@ -21,7 +21,7 @@
 import edu.uci.ics.hyracks.storage.am.common.tuples.TypeAwareTupleWriter;
 
 public class LSMBTreeTupleWriter extends TypeAwareTupleWriter {
-    private final boolean isAntimatter;
+    private boolean isAntimatter;
     private final int numKeyFields;
 
     public LSMBTreeTupleWriter(ITypeTraits[] typeTraits, int numKeyFields, boolean isAntimatter) {
@@ -78,4 +78,9 @@
         // Set leftmost bit to 1.
         targetBuf[targetOff] = (byte) (targetBuf[targetOff] | (1 << 7));
     }
+
+    // Allow using the same writer for both delete and insert tuples
+    public void setAntimatter(boolean isAntimatter) {
+        this.isAntimatter = isAntimatter;
+    }
 }
diff --git a/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/tuples/LSMBTreeTupleWriterFactory.java b/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/tuples/LSMBTreeTupleWriterFactory.java
index 0cbefc9..f184f96 100644
--- a/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/tuples/LSMBTreeTupleWriterFactory.java
+++ b/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/tuples/LSMBTreeTupleWriterFactory.java
@@ -22,13 +22,11 @@
 public class LSMBTreeTupleWriterFactory extends TypeAwareTupleWriterFactory {
 
 	private static final long serialVersionUID = 1L;
-	private final ITypeTraits[] typeTraits;
 	private final int numKeyFields;
 	private final boolean isDelete;
 	
 	public LSMBTreeTupleWriterFactory(ITypeTraits[] typeTraits, int numKeyFields, boolean isDelete) {
 		super(typeTraits);
-		this.typeTraits = typeTraits;
 		this.numKeyFields = numKeyFields;
 		this.isDelete = isDelete;
 	}
diff --git a/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/util/LSMBTreeUtils.java b/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/util/LSMBTreeUtils.java
index 88148c3..6d7564b 100644
--- a/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/util/LSMBTreeUtils.java
+++ b/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/util/LSMBTreeUtils.java
@@ -29,9 +29,14 @@
 import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexMetaDataFrameFactory;
 import edu.uci.ics.hyracks.storage.am.common.frames.LIFOMetaDataFrameFactory;
 import edu.uci.ics.hyracks.storage.am.common.freepage.LinkedListFreePageManagerFactory;
+import edu.uci.ics.hyracks.storage.am.common.tuples.TypeAwareTupleWriterFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.btree.impls.ExternalBTree;
+import edu.uci.ics.hyracks.storage.am.lsm.btree.impls.ExternalBTreeWithBuddy;
 import edu.uci.ics.hyracks.storage.am.lsm.btree.impls.LSMBTree;
 import edu.uci.ics.hyracks.storage.am.lsm.btree.impls.LSMBTreeFileManager;
+import edu.uci.ics.hyracks.storage.am.lsm.btree.impls.LSMBTreeWithBuddyFileManager;
 import edu.uci.ics.hyracks.storage.am.lsm.btree.tuples.LSMBTreeCopyTupleWriterFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.btree.tuples.LSMBTreeRefrencingTupleWriterFactory;
 import edu.uci.ics.hyracks.storage.am.lsm.btree.tuples.LSMBTreeTupleWriterFactory;
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
@@ -81,4 +86,112 @@
                 opTracker, ioScheduler, ioOpCallback, needKeyDupCheck);
         return lsmTree;
     }
+
+    public static ExternalBTree createExternalBTree(FileReference file, IBufferCache diskBufferCache,
+            IFileMapProvider diskFileMapProvider, ITypeTraits[] typeTraits, IBinaryComparatorFactory[] cmpFactories,
+            int[] bloomFilterKeyFields, double bloomFilterFalsePositiveRate, ILSMMergePolicy mergePolicy,
+            ILSMOperationTracker opTracker, ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallback ioOpCallback,
+            int startWithVersion) {
+        LSMBTreeTupleWriterFactory insertTupleWriterFactory = new LSMBTreeTupleWriterFactory(typeTraits,
+                cmpFactories.length, false);
+        LSMBTreeTupleWriterFactory deleteTupleWriterFactory = new LSMBTreeTupleWriterFactory(typeTraits,
+                cmpFactories.length, true);
+        LSMBTreeCopyTupleWriterFactory copyTupleWriterFactory = new LSMBTreeCopyTupleWriterFactory(typeTraits,
+                cmpFactories.length);
+        ITreeIndexFrameFactory insertLeafFrameFactory = new BTreeNSMLeafFrameFactory(insertTupleWriterFactory);
+        ITreeIndexFrameFactory copyTupleLeafFrameFactory = new BTreeNSMLeafFrameFactory(copyTupleWriterFactory);
+        ITreeIndexFrameFactory deleteLeafFrameFactory = new BTreeNSMLeafFrameFactory(deleteTupleWriterFactory);
+        ITreeIndexFrameFactory interiorFrameFactory = new BTreeNSMInteriorFrameFactory(insertTupleWriterFactory);
+        ITreeIndexMetaDataFrameFactory metaFrameFactory = new LIFOMetaDataFrameFactory();
+        IFreePageManagerFactory freePageManagerFactory = new LinkedListFreePageManagerFactory(diskBufferCache,
+                metaFrameFactory);
+        // This is the tuple writer that can do both inserts and deletes
+        LSMBTreeRefrencingTupleWriterFactory referencingTupleWriterFactory = new LSMBTreeRefrencingTupleWriterFactory(
+                typeTraits, cmpFactories.length, false);
+        // This is the leaf frame factory for transaction components since it
+        // can be used for both inserts and deletes
+        ITreeIndexFrameFactory dualLeafFrameFactory = new BTreeNSMLeafFrameFactory(referencingTupleWriterFactory);
+
+        TreeIndexFactory<BTree> diskBTreeFactory = new BTreeFactory(diskBufferCache, diskFileMapProvider,
+                freePageManagerFactory, interiorFrameFactory, copyTupleLeafFrameFactory, cmpFactories,
+                typeTraits.length);
+        TreeIndexFactory<BTree> bulkLoadBTreeFactory = new BTreeFactory(diskBufferCache, diskFileMapProvider,
+                freePageManagerFactory, interiorFrameFactory, insertLeafFrameFactory, cmpFactories, typeTraits.length);
+
+        BloomFilterFactory bloomFilterFactory = new BloomFilterFactory(diskBufferCache, diskFileMapProvider,
+                bloomFilterKeyFields);
+
+        // This is the component factory for transactions
+        TreeIndexFactory<BTree> transactionBTreeFactory = new BTreeFactory(diskBufferCache, diskFileMapProvider,
+                freePageManagerFactory, interiorFrameFactory, dualLeafFrameFactory, cmpFactories, typeTraits.length);
+
+        ILSMIndexFileManager fileNameManager = new LSMBTreeFileManager(diskFileMapProvider, file, diskBTreeFactory);
+
+        // the disk only index uses an empty ArrayList for virtual buffer caches
+        ExternalBTree lsmTree = new ExternalBTree(interiorFrameFactory, insertLeafFrameFactory, deleteLeafFrameFactory,
+                fileNameManager, diskBTreeFactory, bulkLoadBTreeFactory, bloomFilterFactory,
+                bloomFilterFalsePositiveRate, diskFileMapProvider, typeTraits.length, cmpFactories, mergePolicy,
+                opTracker, ioScheduler, ioOpCallback, transactionBTreeFactory, startWithVersion);
+        return lsmTree;
+    }
+
+    public static ExternalBTreeWithBuddy createExternalBTreeWithBuddy(FileReference file, IBufferCache diskBufferCache,
+            IFileMapProvider diskFileMapProvider, ITypeTraits[] typeTraits, IBinaryComparatorFactory[] cmpFactories,
+            double bloomFilterFalsePositiveRate, ILSMMergePolicy mergePolicy, ILSMOperationTracker opTracker,
+            ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallback ioOpCallback, int[] buddyBTreeFields,
+            int startWithVersion) {
+
+        ITypeTraits[] buddyBtreeTypeTraits = new ITypeTraits[buddyBTreeFields.length];
+        IBinaryComparatorFactory[] buddyBtreeCmpFactories = new IBinaryComparatorFactory[buddyBTreeFields.length];
+        for (int i = 0; i < buddyBtreeTypeTraits.length; i++) {
+            buddyBtreeTypeTraits[i] = typeTraits[buddyBTreeFields[i]];
+            buddyBtreeCmpFactories[i] = cmpFactories[buddyBTreeFields[i]];
+        }
+
+        TypeAwareTupleWriterFactory buddyBtreeTupleWriterFactory = new TypeAwareTupleWriterFactory(buddyBtreeTypeTraits);
+        ITreeIndexFrameFactory buddyBtreeInteriorFrameFactory = new BTreeNSMInteriorFrameFactory(
+                buddyBtreeTupleWriterFactory);
+        ITreeIndexFrameFactory buddyBtreeLeafFrameFactory = new BTreeNSMLeafFrameFactory(buddyBtreeTupleWriterFactory);
+
+        LSMBTreeTupleWriterFactory insertTupleWriterFactory = new LSMBTreeTupleWriterFactory(typeTraits,
+                cmpFactories.length, false);
+        LSMBTreeCopyTupleWriterFactory copyTupleWriterFactory = new LSMBTreeCopyTupleWriterFactory(typeTraits,
+                cmpFactories.length);
+        ITreeIndexFrameFactory insertLeafFrameFactory = new BTreeNSMLeafFrameFactory(insertTupleWriterFactory);
+        ITreeIndexFrameFactory copyTupleLeafFrameFactory = new BTreeNSMLeafFrameFactory(copyTupleWriterFactory);
+        ITreeIndexFrameFactory interiorFrameFactory = new BTreeNSMInteriorFrameFactory(insertTupleWriterFactory);
+        ITreeIndexMetaDataFrameFactory metaFrameFactory = new LIFOMetaDataFrameFactory();
+        IFreePageManagerFactory freePageManagerFactory = new LinkedListFreePageManagerFactory(diskBufferCache,
+                metaFrameFactory);
+
+        TreeIndexFactory<BTree> diskBTreeFactory = new BTreeFactory(diskBufferCache, diskFileMapProvider,
+                freePageManagerFactory, interiorFrameFactory, copyTupleLeafFrameFactory, cmpFactories,
+                typeTraits.length);
+
+        TreeIndexFactory<BTree> bulkLoadBTreeFactory = new BTreeFactory(diskBufferCache, diskFileMapProvider,
+                freePageManagerFactory, interiorFrameFactory, insertLeafFrameFactory, cmpFactories, typeTraits.length);
+
+        int[] bloomFilterKeyFields = new int[buddyBtreeCmpFactories.length];
+        for (int i = 0; i < buddyBtreeCmpFactories.length; i++) {
+            bloomFilterKeyFields[i] = i;
+        }
+        BloomFilterFactory bloomFilterFactory = new BloomFilterFactory(diskBufferCache, diskFileMapProvider,
+                bloomFilterKeyFields);
+
+        // buddy b-tree factory
+        TreeIndexFactory<BTree> diskBuddyBTreeFactory = new BTreeFactory(diskBufferCache, diskFileMapProvider,
+                freePageManagerFactory, buddyBtreeInteriorFrameFactory, buddyBtreeLeafFrameFactory,
+                buddyBtreeCmpFactories, buddyBtreeTypeTraits.length);
+
+        ILSMIndexFileManager fileNameManager = new LSMBTreeWithBuddyFileManager(diskFileMapProvider, file,
+                diskBTreeFactory, diskBuddyBTreeFactory);
+
+        // the disk only index uses an empty ArrayList for virtual buffer caches
+        ExternalBTreeWithBuddy lsmTree = new ExternalBTreeWithBuddy(interiorFrameFactory, insertLeafFrameFactory,
+                buddyBtreeLeafFrameFactory, diskBufferCache, fileNameManager, bulkLoadBTreeFactory, diskBTreeFactory,
+                diskBuddyBTreeFactory, bloomFilterFactory, diskFileMapProvider, bloomFilterFalsePositiveRate,
+                mergePolicy, opTracker, ioScheduler, ioOpCallback, cmpFactories, buddyBtreeCmpFactories,
+                buddyBTreeFields, startWithVersion);
+        return lsmTree;
+    }
 }
diff --git a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/api/ILSMIndexFileManager.java b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/api/ILSMIndexFileManager.java
index 6d460d6..ba3d9e3 100644
--- a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/api/ILSMIndexFileManager.java
+++ b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/api/ILSMIndexFileManager.java
@@ -15,6 +15,7 @@
 
 package edu.uci.ics.hyracks.storage.am.lsm.common.api;
 
+import java.io.IOException;
 import java.util.Comparator;
 import java.util.List;
 
@@ -46,4 +47,31 @@
     public List<LSMComponentFileReferences> cleanupAndGetValidFiles() throws HyracksDataException, IndexException;
 
     public Comparator<String> getFileNameComparator();
+
+    /**
+     * @return delete existing transaction disk component file reference
+     * @throws HyracksDataException
+     */
+    public void deleteTransactionFiles() throws HyracksDataException;
+
+    /**
+     * Rename files of a transaction removing the transaction prefix and return the component file reference in order to be committed
+     * 
+     * @return the renamed component file references
+     * @throws HyracksDataException
+     */
+    public LSMComponentFileReferences getTransactionFileReferenceForCommit() throws HyracksDataException;
+
+    /**
+     * Recover transaction files without returning them
+     * 
+     * @throws HyracksDataException
+     */
+    public void recoverTransaction() throws HyracksDataException;
+
+    /**
+     * @return a reference to the transaction disk component file reference
+     * @throws IOException
+     */
+    public LSMComponentFileReferences getNewTransactionFileReference() throws IOException;
 }
diff --git a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/api/ILSMIndexInternal.java b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/api/ILSMIndexInternal.java
index 6020093..b8f590f 100644
--- a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/api/ILSMIndexInternal.java
+++ b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/api/ILSMIndexInternal.java
@@ -45,9 +45,9 @@
 
     public ILSMComponent merge(ILSMIOOperation operation) throws HyracksDataException, IndexException;
 
-    public void addComponent(ILSMComponent index);
+    public void addComponent(ILSMComponent index) throws HyracksDataException;
 
-    public void subsumeMergedComponents(ILSMComponent newComponent, List<ILSMComponent> mergedComponents);
+    public void subsumeMergedComponents(ILSMComponent newComponent, List<ILSMComponent> mergedComponents) throws HyracksDataException;
 
     public void changeMutableComponent();
 
@@ -64,5 +64,4 @@
     public void getOperationalComponents(ILSMIndexOperationContext ctx);
 
     public void markAsValid(ILSMComponent lsmComponent) throws HyracksDataException;
-
 }
diff --git a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/api/ITwoPCIndex.java b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/api/ITwoPCIndex.java
new file mode 100644
index 0000000..cc4d65e
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/api/ITwoPCIndex.java
@@ -0,0 +1,73 @@
+/*
+ * 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.hyracks.storage.am.lsm.common.api;
+
+import java.util.List;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexBulkLoader;
+import edu.uci.ics.hyracks.storage.am.common.api.ISearchOperationCallback;
+import edu.uci.ics.hyracks.storage.am.common.api.IndexException;
+import edu.uci.ics.hyracks.storage.am.common.api.TreeIndexException;
+
+// An interface containing the new methods introduced for bulk transactions
+public interface ITwoPCIndex {
+    /**
+     * This function is used to create a BulkLoader for a transaction that is capable of insertions and deletions
+     * and the bulk loaded component is hidden from the index
+     */
+    public IIndexBulkLoader createTransactionBulkLoader(float fillLevel, boolean verifyInput, long numElementsHint,
+            boolean checkIfEmptyIndex) throws TreeIndexException;
+    
+    /**
+     * This function is used to commit the previous transaction if it was resulted in creating any components
+     */
+    public void commitTransaction() throws TreeIndexException, HyracksDataException, IndexException;
+    
+    /**
+     * This function is used to abort the last transaction
+     */
+    public void abortTransaction() throws TreeIndexException;
+    
+    /**
+     * This function is used to recover a transaction if the system crashed after the decision to commit
+     */
+    public void recoverTransaction() throws TreeIndexException;
+    
+    /**
+     * This function is used to add the committed disk component to the appropriate list and reflect the changes
+     */
+    public void commitTransactionDiskComponent(ILSMComponent newComponent) throws IndexException, HyracksDataException;
+    
+    /**
+     * This function is used to create a version specific accessor to search a specific version
+     */
+    public ILSMIndexAccessorInternal createAccessor(ISearchOperationCallback searchCallback, int targetIndexVersion) throws HyracksDataException;
+    
+    /**
+     * This function is used to get the first components list
+     */
+    public List<ILSMComponent> getFirstComponentList();
+    
+    /**
+     * This function is used to get teh second components list
+     */
+    public List<ILSMComponent> getSecondComponentList();
+    
+    /**
+     * This function is used to get the current version id of the index
+     */
+    public int getCurrentVersion();
+}
diff --git a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/AbortRecoverLSMIndexFileManager.java b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/AbortRecoverLSMIndexFileManager.java
new file mode 100644
index 0000000..122f63e
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/AbortRecoverLSMIndexFileManager.java
@@ -0,0 +1,23 @@
+/*
+ * 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.hyracks.storage.am.lsm.common.impls;
+
+import edu.uci.ics.hyracks.api.io.FileReference;
+
+public class AbortRecoverLSMIndexFileManager extends AbstractLSMIndexFileManager {
+    public AbortRecoverLSMIndexFileManager(FileReference file) {
+        super(null, file, null);
+    }
+}
diff --git a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/AbstractLSMIndex.java b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/AbstractLSMIndex.java
index 3d7512f..e5fcf0b 100644
--- a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/AbstractLSMIndex.java
+++ b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/AbstractLSMIndex.java
@@ -27,7 +27,6 @@
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMComponent;
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMHarness;
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
-import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackProvider;
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndexFileManager;
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndexInternal;
@@ -84,6 +83,26 @@
         }
     }
 
+    // The constructor used by external indexes
+    public AbstractLSMIndex(IBufferCache diskBufferCache, ILSMIndexFileManager fileManager,
+            IFileMapProvider diskFileMapProvider, double bloomFilterFalsePositiveRate, ILSMMergePolicy mergePolicy,
+            ILSMOperationTracker opTracker, ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallback ioOpCallback) {
+        this.diskBufferCache = diskBufferCache;
+        this.diskFileMapProvider = diskFileMapProvider;
+        this.fileManager = fileManager;
+        this.bloomFilterFalsePositiveRate = bloomFilterFalsePositiveRate;
+        this.ioScheduler = ioScheduler;
+        this.ioOpCallback = ioOpCallback;
+        lsmHarness = new ExternalIndexHarness(this, mergePolicy, opTracker);
+        isActivated = false;
+        diskComponents = new LinkedList<ILSMComponent>();
+        // Memory related objects are nulled
+        this.virtualBufferCaches = null;
+        memoryComponents = null;
+        currentMutableComponentId = null;
+        flushRequests = null;
+    }
+
     protected void forceFlushDirtyPages(ITreeIndex treeIndex) throws HyracksDataException {
         int fileId = treeIndex.getFileId();
         IBufferCache bufferCache = treeIndex.getBufferCache();
@@ -147,12 +166,13 @@
     }
 
     @Override
-    public void addComponent(ILSMComponent c) {
+    public void addComponent(ILSMComponent c) throws HyracksDataException {
         diskComponents.add(0, c);
     }
 
     @Override
-    public void subsumeMergedComponents(ILSMComponent newComponent, List<ILSMComponent> mergedComponents) {
+    public void subsumeMergedComponents(ILSMComponent newComponent, List<ILSMComponent> mergedComponents)
+            throws HyracksDataException {
         int swapIndex = diskComponents.indexOf(mergedComponents.get(0));
         diskComponents.removeAll(mergedComponents);
         diskComponents.add(swapIndex, newComponent);
@@ -215,4 +235,9 @@
     public String toString() {
         return "LSMIndex [" + fileManager.getBaseDir() + "]";
     }
+    
+    @Override
+    public boolean hasMemoryComponents() {
+        return true;
+    }
 }
diff --git a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/AbstractLSMIndexFileManager.java b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/AbstractLSMIndexFileManager.java
index a6425ce..1676560 100644
--- a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/AbstractLSMIndexFileManager.java
+++ b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/AbstractLSMIndexFileManager.java
@@ -17,6 +17,9 @@
 
 import java.io.File;
 import java.io.FilenameFilter;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.nio.file.Paths;
 import java.text.Format;
 import java.text.SimpleDateFormat;
 import java.util.ArrayList;
@@ -41,6 +44,7 @@
 
     protected static final String SPLIT_STRING = "_";
     protected static final String BLOOM_FILTER_STRING = "f";
+    protected static final String TRANSACTION_PREFIX = ".T";
 
     protected final IFileMapProvider fileMapProvider;
 
@@ -256,6 +260,22 @@
         return baseDir;
     }
 
+    public void recoverTransaction() throws HyracksDataException {
+        File dir = new File(baseDir);
+        String[] files = dir.list(transactionFileNameFilter);
+        try {
+            if (files.length == 0) {
+                // Do nothing
+            } else if (files.length > 1) {
+                throw new HyracksDataException("Found more than one transaction");
+            } else {
+                Files.delete(Paths.get(dir.getPath() + File.separator + files[0]));
+            }
+        } catch (IOException e) {
+            throw new HyracksDataException("Failed to recover transaction", e);
+        }
+    }
+
     protected class ComparableFileName implements Comparable<ComparableFileName> {
         public final FileReference fileRef;
         public final String fullPath;
@@ -291,4 +311,90 @@
             return -a.interval[1].compareTo(b.interval[1]);
         }
     }
+
+    // This function is used to delete transaction files for aborted transactions
+    @Override
+    public void deleteTransactionFiles() throws HyracksDataException {
+        File dir = new File(baseDir);
+        String[] files = dir.list(transactionFileNameFilter);
+        if (files.length == 0) {
+            // Do nothing
+        } else if (files.length > 1) {
+            throw new HyracksDataException("Found more than one transaction");
+        } else {
+            //create transaction filter
+            FilenameFilter transactionFilter = createTransactionFilter(files[0], true);
+            String[] componentsFiles = dir.list(transactionFilter);
+            for (String fileName : componentsFiles) {
+                try {
+                    String absFileName = dir.getPath() + File.separator + fileName;
+                    Files.delete(Paths.get(absFileName));
+                } catch (IOException e) {
+                    throw new HyracksDataException("Failed to delete transaction files", e);
+                }
+            }
+            // delete the txn lock file
+            String absFileName = dir.getPath() + File.separator + files[0];
+            try {
+                Files.delete(Paths.get(absFileName));
+            } catch (IOException e) {
+                throw new HyracksDataException("Failed to delete transaction files", e);
+            }
+        }
+    }
+
+    @Override
+    public LSMComponentFileReferences getNewTransactionFileReference() throws IOException {
+        return null;
+    }
+
+    @Override
+    public LSMComponentFileReferences getTransactionFileReferenceForCommit() throws HyracksDataException {
+        return null;
+    }
+
+    protected static FilenameFilter transactionFileNameFilter = new FilenameFilter() {
+        public boolean accept(File dir, String name) {
+            return name.startsWith(".T");
+        }
+    };
+
+    protected static FilenameFilter dummyFilter = new FilenameFilter() {
+        public boolean accept(File dir, String name) {
+            return true;
+        }
+    };
+
+    protected static FilenameFilter createTransactionFilter(String transactionFileName, final boolean inclusive) {
+        final String timeStamp = transactionFileName.substring(transactionFileName.indexOf(TRANSACTION_PREFIX)
+                + TRANSACTION_PREFIX.length());
+        return new FilenameFilter() {
+            public boolean accept(File dir, String name) {
+                if (inclusive) {
+                    return name.startsWith(timeStamp);
+                } else {
+                    return !name.startsWith(timeStamp);
+                }
+            }
+        };
+    }
+
+    protected FilenameFilter getTransactionFileFilter(boolean inclusive) {
+        File dir = new File(baseDir);
+        String[] files = dir.list(transactionFileNameFilter);
+        if (files.length == 0) {
+            return dummyFilter;
+        } else {
+            return createTransactionFilter(files[0], inclusive);
+        }
+    }
+
+    protected FilenameFilter getCompoundFilter(final FilenameFilter filter1, final FilenameFilter filter2) {
+        return new FilenameFilter() {
+            @Override
+            public boolean accept(File dir, String name) {
+                return (filter1.accept(dir, name) && filter2.accept(dir, name));
+            }
+        };
+    }
 }
diff --git a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/ExternalIndexHarness.java b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/ExternalIndexHarness.java
new file mode 100644
index 0000000..8c763cb
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/ExternalIndexHarness.java
@@ -0,0 +1,338 @@
+/*
+ * 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.hyracks.storage.am.lsm.common.impls;
+
+import java.util.List;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexCursor;
+import edu.uci.ics.hyracks.storage.am.common.api.ISearchPredicate;
+import edu.uci.ics.hyracks.storage.am.common.api.IndexException;
+import edu.uci.ics.hyracks.storage.am.common.ophelpers.IndexOperation;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMComponent;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndexInternal;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndexOperationContext;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMMergePolicy;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTracker;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ITwoPCIndex;
+
+public class ExternalIndexHarness extends LSMHarness {
+    private static final Logger LOGGER = Logger.getLogger(ExternalIndexHarness.class.getName());
+
+
+    public ExternalIndexHarness(ILSMIndexInternal lsmIndex, ILSMMergePolicy mergePolicy, ILSMOperationTracker opTracker) {
+        super(lsmIndex,mergePolicy,opTracker);
+    }
+
+    @Override
+    protected boolean getAndEnterComponents(ILSMIndexOperationContext ctx, LSMOperationType opType, boolean isTryOperation)
+            throws HyracksDataException {
+        synchronized (opTracker) {
+            while (true) {
+                lsmIndex.getOperationalComponents(ctx);
+                // Before entering the components, prune those corner cases that indeed should not proceed.
+                switch (opType) {
+                    case MERGE:
+                        if (ctx.getComponentHolder().size() < 2) {
+                            // There is only a single component. There is nothing to merge.
+                            return false;
+                        }
+                    default:
+                        break;
+                }
+                if (enterComponents(ctx, opType)) {
+                    return true;
+                } else if (isTryOperation) {
+                    return false;
+                }
+            }
+        }
+    }
+
+    @Override
+    protected boolean enterComponents(ILSMIndexOperationContext ctx, LSMOperationType opType) throws HyracksDataException {
+        List<ILSMComponent> components = ctx.getComponentHolder();
+        int numEntered = 0;
+        boolean entranceSuccessful = false;
+        try {
+            for (ILSMComponent c : components) {
+                if (!c.threadEnter(opType, false)) {
+                    break;
+                }
+                numEntered++;
+            }
+            entranceSuccessful = numEntered == components.size();
+        } finally {
+            if (!entranceSuccessful) {
+                for (ILSMComponent c : components) {
+                    if (numEntered == 0) {
+                        break;
+                    }
+                    c.threadExit(opType, true, false);
+                    numEntered--;
+                }
+                return false;
+            }
+        }
+        // Check if there is any action that is needed to be taken based on the operation type
+        switch (opType) {
+            case MERGE:
+                lsmIndex.getIOOperationCallback().beforeOperation(LSMOperationType.MERGE);
+            default:
+                break;
+        }
+        opTracker.beforeOperation(lsmIndex, opType, ctx.getSearchOperationCallback(), ctx.getModificationCallback());
+        return true;
+    }
+
+    private void exitComponents(ILSMIndexOperationContext ctx, LSMOperationType opType, ILSMComponent newComponent,
+            boolean failedOperation) throws HyracksDataException, IndexException {
+        synchronized (opTracker) {
+            try {
+                // First check if there is any action that is needed to be taken based on the state of each component.
+                for (ILSMComponent c : ctx.getComponentHolder()) {
+                    c.threadExit(opType, failedOperation, false);
+                    switch (c.getState()) {
+                        case INACTIVE:
+                            ((AbstractDiskLSMComponent) c).destroy();
+                            break;
+                        default:
+                            break;
+                    }
+                }
+                // Then, perform any action that is needed to be taken based on the operation type.
+                switch (opType) {
+                    case MERGE:
+                        // newComponent is null if the merge op. was not performed.
+                        if (newComponent != null) {
+                            beforeSubsumeMergedComponents(newComponent, ctx.getComponentHolder());
+                            lsmIndex.subsumeMergedComponents(newComponent, ctx.getComponentHolder());
+                            mergePolicy.diskComponentAdded(lsmIndex, fullMergeIsRequested.get());
+                        }
+                        break;
+                    default:
+                        break;
+                }
+            } finally {
+                opTracker.afterOperation(lsmIndex, opType, ctx.getSearchOperationCallback(),
+                        ctx.getModificationCallback());
+            }
+        }
+    }
+
+    @Override
+    public void forceModify(ILSMIndexOperationContext ctx, ITupleReference tuple) throws HyracksDataException,
+            IndexException {
+        throw new IndexException("2PC LSM Inedx doesn't support modify");
+    }
+
+    @Override
+    public boolean modify(ILSMIndexOperationContext ctx, boolean tryOperation, ITupleReference tuple)
+            throws HyracksDataException, IndexException {
+        throw new IndexException("2PC LSM Inedx doesn't support modify");
+    }
+
+    @Override
+    public void search(ILSMIndexOperationContext ctx, IIndexCursor cursor, ISearchPredicate pred)
+            throws HyracksDataException, IndexException {
+        LSMOperationType opType = LSMOperationType.SEARCH;
+        getAndEnterComponents(ctx, opType, false);
+        try {
+            lsmIndex.search(ctx, cursor, pred);
+        } catch (HyracksDataException | IndexException e) {
+            exitComponents(ctx, opType, null, true);
+            throw e;
+        }
+    }
+
+    @Override
+    public void endSearch(ILSMIndexOperationContext ctx) throws HyracksDataException {
+        if (ctx.getOperation() == IndexOperation.SEARCH) {
+            try {
+                exitComponents(ctx, LSMOperationType.SEARCH, null, false);
+            } catch (IndexException e) {
+                throw new HyracksDataException(e);
+            }
+        }
+    }
+
+    @Override
+    public void scheduleMerge(ILSMIndexOperationContext ctx, ILSMIOOperationCallback callback)
+            throws HyracksDataException, IndexException {
+        if (!getAndEnterComponents(ctx, LSMOperationType.MERGE, true)) {
+            callback.afterFinalize(LSMOperationType.MERGE, null);
+            return;
+        }
+        lsmIndex.scheduleMerge(ctx, callback);
+    }
+
+    @Override
+    public void scheduleFullMerge(ILSMIndexOperationContext ctx, ILSMIOOperationCallback callback)
+            throws HyracksDataException, IndexException {
+        fullMergeIsRequested.set(true);
+        if (!getAndEnterComponents(ctx, LSMOperationType.MERGE, true)) {
+            // If the merge cannot be scheduled because there is already an ongoing merge on subset/all of the components, then
+            // whenever the current merge has finished, it will schedule the full merge again.
+            callback.afterFinalize(LSMOperationType.MERGE, null);
+            return;
+        }
+        fullMergeIsRequested.set(false);
+        lsmIndex.scheduleMerge(ctx, callback);
+    }
+
+    @Override
+    public void merge(ILSMIndexOperationContext ctx, ILSMIOOperation operation) throws HyracksDataException,
+            IndexException {
+        if (LOGGER.isLoggable(Level.INFO)) {
+            LOGGER.info("Started a merge operation for index: " + lsmIndex + " ...");
+        }
+
+        ILSMComponent newComponent = null;
+        try {
+            newComponent = lsmIndex.merge(operation);
+            operation.getCallback().afterOperation(LSMOperationType.MERGE, ctx.getComponentHolder(), newComponent);
+            lsmIndex.markAsValid(newComponent);
+        } finally {
+            exitComponents(ctx, LSMOperationType.MERGE, newComponent, false);
+            operation.getCallback().afterFinalize(LSMOperationType.MERGE, newComponent);
+        }
+        if (LOGGER.isLoggable(Level.INFO)) {
+            LOGGER.info("Finished the merge operation for index: " + lsmIndex);
+        }
+    }
+
+    @Override
+    public void addBulkLoadedComponent(ILSMComponent c) throws HyracksDataException, IndexException {
+        lsmIndex.markAsValid(c);
+        lsmIndex.addComponent(c);
+        // Enter the component
+        enterComponent(c);
+        mergePolicy.diskComponentAdded(lsmIndex, false);
+    }
+
+    // Three differences from  addBulkLoadedComponent
+    // 1. this needs synchronization since others might be accessing the index (specifically merge operations that might change the lists of components)
+    // 2. the actions taken by the index itself are different
+    // 3. the component has already been marked valid by the bulk update operation
+    public void addTransactionComponents(ILSMComponent newComponent) throws HyracksDataException, IndexException {
+        ITwoPCIndex index = (ITwoPCIndex) lsmIndex;
+        synchronized (opTracker) {
+            List<ILSMComponent> newerList;
+            List<ILSMComponent> olderList;
+            if (index.getCurrentVersion() == 0) {
+                newerList = index.getFirstComponentList();
+                olderList = index.getSecondComponentList();
+            } else {
+                newerList = index.getSecondComponentList();
+                olderList = index.getFirstComponentList();
+            }
+            // Exit components in old version of the index so they are ready to be
+            // deleted if they are not needed anymore
+            for (ILSMComponent c : olderList) {
+                exitComponent(c);
+            }
+            // Enter components in the newer list
+            for (ILSMComponent c : newerList) {
+                enterComponent(c);
+            }
+            if (newComponent != null) {
+                // Enter new component
+                enterComponent(newComponent);
+            }
+            index.commitTransactionDiskComponent(newComponent);
+            mergePolicy.diskComponentAdded(lsmIndex, fullMergeIsRequested.get());
+        }
+    }
+
+    @Override
+    public void scheduleFlush(ILSMIndexOperationContext ctx, ILSMIOOperationCallback callback)
+            throws HyracksDataException {
+        callback.afterFinalize(LSMOperationType.FLUSH, null);
+    }
+
+    @Override
+    public void flush(ILSMIndexOperationContext ctx, ILSMIOOperation operation) throws HyracksDataException,
+            IndexException {
+    }
+
+    @Override
+    public ILSMOperationTracker getOperationTracker() {
+        return opTracker;
+    }
+
+    public void beforeSubsumeMergedComponents(ILSMComponent newComponent, List<ILSMComponent> mergedComponents)
+            throws HyracksDataException {
+        ITwoPCIndex index = (ITwoPCIndex) lsmIndex;
+        // check if merge will affect the first list
+        if (index.getFirstComponentList().containsAll(mergedComponents)) {
+            // exit un-needed components
+            for (ILSMComponent c : mergedComponents) {
+                exitComponent(c);
+            }
+            // enter new component
+            enterComponent(newComponent);
+        }
+        // check if merge will affect the second list
+        if (index.getSecondComponentList().containsAll(mergedComponents)) {
+            // exit un-needed components
+            for (ILSMComponent c : mergedComponents) {
+                exitComponent(c);
+            }
+            // enter new component
+            enterComponent(newComponent);
+        }
+    }
+
+    // The two methods: enterComponent and exitComponent are used to control
+    // when components are to be deleted from disk
+    private void enterComponent(ILSMComponent diskComponent) throws HyracksDataException {
+        diskComponent.threadEnter(LSMOperationType.SEARCH, false);
+    }
+
+    private void exitComponent(ILSMComponent diskComponent) throws HyracksDataException {
+        diskComponent.threadExit(LSMOperationType.SEARCH, false, false);
+        if (diskComponent.getState() == ILSMComponent.ComponentState.INACTIVE) {
+            ((AbstractDiskLSMComponent) diskComponent).destroy();
+        }
+    }
+
+    public void indexFirstTimeActivated() throws HyracksDataException {
+        ITwoPCIndex index = (ITwoPCIndex) lsmIndex;
+        // Enter disk components <-- To avoid deleting them when they are
+        // still needed-->
+        for (ILSMComponent c : index.getFirstComponentList()) {
+            enterComponent(c);
+        }
+        for (ILSMComponent c : index.getSecondComponentList()) {
+            enterComponent(c);
+        }
+    }
+
+    public void indexClear() throws HyracksDataException {
+        ITwoPCIndex index = (ITwoPCIndex) lsmIndex;
+        for (ILSMComponent c : index.getFirstComponentList()) {
+            exitComponent(c);
+        }
+        for (ILSMComponent c : index.getSecondComponentList()) {
+            exitComponent(c);
+        }
+    }
+
+}
diff --git a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/LSMHarness.java b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/LSMHarness.java
index 8420c73..a88b982 100644
--- a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/LSMHarness.java
+++ b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/LSMHarness.java
@@ -39,10 +39,10 @@
 public class LSMHarness implements ILSMHarness {
     private static final Logger LOGGER = Logger.getLogger(LSMHarness.class.getName());
 
-    private final ILSMIndexInternal lsmIndex;
-    private final ILSMMergePolicy mergePolicy;
-    private final ILSMOperationTracker opTracker;
-    private final AtomicBoolean fullMergeIsRequested;
+    protected final ILSMIndexInternal lsmIndex;
+    protected final ILSMMergePolicy mergePolicy;
+    protected final ILSMOperationTracker opTracker;
+    protected final AtomicBoolean fullMergeIsRequested;
 
     public LSMHarness(ILSMIndexInternal lsmIndex, ILSMMergePolicy mergePolicy, ILSMOperationTracker opTracker) {
         this.lsmIndex = lsmIndex;
@@ -51,7 +51,7 @@
         fullMergeIsRequested = new AtomicBoolean();
     }
 
-    private boolean getAndEnterComponents(ILSMIndexOperationContext ctx, LSMOperationType opType, boolean isTryOperation)
+    protected boolean getAndEnterComponents(ILSMIndexOperationContext ctx, LSMOperationType opType, boolean isTryOperation)
             throws HyracksDataException {
         synchronized (opTracker) {
             while (true) {
@@ -90,7 +90,7 @@
         }
     }
 
-    private boolean enterComponents(ILSMIndexOperationContext ctx, LSMOperationType opType) throws HyracksDataException {
+    protected boolean enterComponents(ILSMIndexOperationContext ctx, LSMOperationType opType) throws HyracksDataException {
         List<ILSMComponent> components = ctx.getComponentHolder();
         int numEntered = 0;
         boolean entranceSuccessful = false;
@@ -217,7 +217,7 @@
         return modify(ctx, tryOperation, tuple, opType);
     }
 
-    private boolean modify(ILSMIndexOperationContext ctx, boolean tryOperation, ITupleReference tuple,
+    protected boolean modify(ILSMIndexOperationContext ctx, boolean tryOperation, ITupleReference tuple,
             LSMOperationType opType) throws HyracksDataException, IndexException {
         if (!getAndEnterComponents(ctx, opType, tryOperation)) {
             return false;
diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndex.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndex.java
index ff0d9e3..44df9af 100644
--- a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndex.java
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndex.java
@@ -822,4 +822,9 @@
     public String toString() {
         return "LSMInvertedIndex [" + fileManager.getBaseDir() + "]";
     }
+    
+    @Override
+    public boolean hasMemoryComponents() {
+        return true;
+    }
 }
diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/inmemory/InMemoryInvertedIndex.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/inmemory/InMemoryInvertedIndex.java
index c70c2d5..f4a3916 100644
--- a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/inmemory/InMemoryInvertedIndex.java
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/inmemory/InMemoryInvertedIndex.java
@@ -202,4 +202,9 @@
     public IBinaryComparatorFactory[] getTokenCmpFactories() {
         return tokenCmpFactories;
     }
+
+    @Override
+    public boolean hasMemoryComponents() {
+        return true;
+    }
 }
diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndex.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndex.java
index 6aa3253..1684cf0 100644
--- a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndex.java
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndex.java
@@ -673,4 +673,9 @@
     public IBinaryComparatorFactory[] getTokenCmpFactories() {
         return tokenCmpFactories;
     }
+    
+    @Override
+    public boolean hasMemoryComponents() {
+        return true;
+    }
 }
diff --git a/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/dataflow/ExternalRTreeDataflowHelper.java b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/dataflow/ExternalRTreeDataflowHelper.java
new file mode 100644
index 0000000..7b823b3
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/dataflow/ExternalRTreeDataflowHelper.java
@@ -0,0 +1,109 @@
+/*
+ * 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.hyracks.storage.am.lsm.rtree.dataflow;
+
+import java.util.List;
+
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.ILinearizeComparatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.io.FileReference;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndex;
+import edu.uci.ics.hyracks.storage.am.common.api.IPrimitiveValueProviderFactory;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
+import edu.uci.ics.hyracks.storage.am.common.api.TreeIndexException;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexOperatorDescriptor;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMMergePolicy;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTracker;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerProvider;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.IVirtualBufferCache;
+import edu.uci.ics.hyracks.storage.am.lsm.rtree.utils.LSMRTreeUtils;
+import edu.uci.ics.hyracks.storage.am.rtree.frames.RTreePolicyType;
+import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
+import edu.uci.ics.hyracks.storage.common.file.IFileMapProvider;
+
+public class ExternalRTreeDataflowHelper extends LSMRTreeDataflowHelper {
+
+    private int version;
+
+    public ExternalRTreeDataflowHelper(IIndexOperatorDescriptor opDesc, IHyracksTaskContext ctx, int partition,
+            IBinaryComparatorFactory[] btreeComparatorFactories,
+            IPrimitiveValueProviderFactory[] valueProviderFactories, RTreePolicyType rtreePolicyType,
+            ILSMMergePolicy mergePolicy, ILSMOperationTrackerProvider opTrackerFactory,
+            ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallbackFactory ioOpCallbackFactory,
+            ILinearizeComparatorFactory linearizeCmpFactory, int[] btreeFields, int version) {
+        super(opDesc, ctx, partition, null, btreeComparatorFactories, valueProviderFactories, rtreePolicyType,
+                mergePolicy, opTrackerFactory, ioScheduler, ioOpCallbackFactory, linearizeCmpFactory, btreeFields);
+        this.version = version;
+    }
+
+    public ExternalRTreeDataflowHelper(IIndexOperatorDescriptor opDesc, IHyracksTaskContext ctx, int partition,
+            double bloomFilterFalsePositiveRate, IBinaryComparatorFactory[] btreeComparatorFactories,
+            IPrimitiveValueProviderFactory[] valueProviderFactories, RTreePolicyType rtreePolicyType,
+            ILSMMergePolicy mergePolicy, ILSMOperationTrackerProvider opTrackerFactory,
+            ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallbackFactory ioOpCallbackFactory,
+            ILinearizeComparatorFactory linearizeCmpFactory, int[] btreeFields, int version) {
+        super(opDesc, ctx, partition, null, bloomFilterFalsePositiveRate, btreeComparatorFactories,
+                valueProviderFactories, rtreePolicyType, mergePolicy, opTrackerFactory, ioScheduler,
+                ioOpCallbackFactory, linearizeCmpFactory, btreeFields);
+        this.version = version;
+    }
+
+    @Override
+    public IIndex getIndexInstance() {
+        if (index != null)
+            return index;
+        synchronized (lcManager) {
+            long resourceID;
+            try {
+                resourceID = getResourceID();
+            } catch (HyracksDataException e) {
+                return null;
+            }
+            try {
+                index = lcManager.getIndex(resourceID);
+            } catch (HyracksDataException e) {
+                return null;
+            }
+        }
+        return index;
+    }
+
+    @Override
+    protected ITreeIndex createLSMTree(List<IVirtualBufferCache> virtualBufferCaches, FileReference file,
+            IBufferCache diskBufferCache, IFileMapProvider diskFileMapProvider, ITypeTraits[] typeTraits,
+            IBinaryComparatorFactory[] rtreeCmpFactories, IBinaryComparatorFactory[] btreeCmpFactories,
+            ILSMOperationTracker opTracker, IPrimitiveValueProviderFactory[] valueProviderFactories,
+            RTreePolicyType rtreePolicyType, ILinearizeComparatorFactory linearizeCmpFactory)
+            throws HyracksDataException {
+        try {
+            return LSMRTreeUtils.createExternalRTree(file, diskBufferCache, diskFileMapProvider, typeTraits,
+                    rtreeCmpFactories, btreeCmpFactories, valueProviderFactories, rtreePolicyType,
+                    bloomFilterFalsePositiveRate, mergePolicy, opTracker, ioScheduler,
+                    ioOpCallbackFactory.createIOOperationCallback(), linearizeCmpFactory, btreeFields, version);
+        } catch (TreeIndexException e) {
+            throw new HyracksDataException(e);
+        }
+    }
+
+    public int getTargetVersion() {
+        return version;
+    }
+
+}
diff --git a/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/dataflow/ExternalRTreeDataflowHelperFactory.java b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/dataflow/ExternalRTreeDataflowHelperFactory.java
new file mode 100644
index 0000000..81b2582
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/dataflow/ExternalRTreeDataflowHelperFactory.java
@@ -0,0 +1,56 @@
+/*
+ * 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.hyracks.storage.am.lsm.rtree.dataflow;
+
+import java.util.Map;
+
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.ILinearizeComparatorFactory;
+import edu.uci.ics.hyracks.storage.am.common.api.IPrimitiveValueProviderFactory;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexOperatorDescriptor;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.IndexDataflowHelper;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationSchedulerProvider;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMMergePolicyFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerProvider;
+import edu.uci.ics.hyracks.storage.am.rtree.frames.RTreePolicyType;
+
+public class ExternalRTreeDataflowHelperFactory extends LSMRTreeDataflowHelperFactory {
+
+    private static final long serialVersionUID = 1L;
+    private int version;
+
+    public ExternalRTreeDataflowHelperFactory(IPrimitiveValueProviderFactory[] valueProviderFactories,
+            RTreePolicyType rtreePolicyType, IBinaryComparatorFactory[] btreeComparatorFactories,
+            ILSMMergePolicyFactory mergePolicyFactory, Map<String, String> mergePolicyProperties,
+            ILSMOperationTrackerProvider opTrackerFactory, ILSMIOOperationSchedulerProvider ioSchedulerProvider,
+            ILSMIOOperationCallbackFactory ioOpCallbackFactory, ILinearizeComparatorFactory linearizeCmpFactory,
+            double bloomFilterFalsePositiveRate, int[] btreeFields, int version) {
+        super(valueProviderFactories, rtreePolicyType, btreeComparatorFactories, null, mergePolicyFactory,
+                mergePolicyProperties, opTrackerFactory, ioSchedulerProvider, ioOpCallbackFactory, linearizeCmpFactory,
+                bloomFilterFalsePositiveRate, btreeFields);
+        this.version = version;
+    }
+
+    @Override
+    public IndexDataflowHelper createIndexDataflowHelper(IIndexOperatorDescriptor opDesc, IHyracksTaskContext ctx,
+            int partition) {
+        return new ExternalRTreeDataflowHelper(opDesc, ctx, partition, bloomFilterFalsePositiveRate,
+                btreeComparatorFactories, valueProviderFactories, rtreePolicyType,
+                mergePolicyFactory.createMergePolicy(mergePolicyProperties), opTrackerFactory,
+                ioSchedulerProvider.getIOScheduler(ctx), ioOpCallbackFactory, linearizeCmpFactory, btreeFields, version);
+    }
+}
diff --git a/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeDataflowHelper.java b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeDataflowHelper.java
index 8d6c0da..f316503 100644
--- a/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeDataflowHelper.java
+++ b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeDataflowHelper.java
@@ -39,7 +39,7 @@
 import edu.uci.ics.hyracks.storage.common.file.IFileMapProvider;
 
 public class LSMRTreeDataflowHelper extends AbstractLSMRTreeDataflowHelper {
-    private int[] btreeFields;
+    protected int[] btreeFields;
 
     public LSMRTreeDataflowHelper(IIndexOperatorDescriptor opDesc, IHyracksTaskContext ctx, int partition,
             List<IVirtualBufferCache> virtualBufferCaches, IBinaryComparatorFactory[] btreeComparatorFactories,
diff --git a/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeDataflowHelperFactory.java b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeDataflowHelperFactory.java
index beb6c20..7548398 100644
--- a/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeDataflowHelperFactory.java
+++ b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeDataflowHelperFactory.java
@@ -35,11 +35,11 @@
 
     private static final long serialVersionUID = 1L;
 
-    private final IBinaryComparatorFactory[] btreeComparatorFactories;
-    private final IPrimitiveValueProviderFactory[] valueProviderFactories;
-    private final RTreePolicyType rtreePolicyType;
-    private final ILinearizeComparatorFactory linearizeCmpFactory;
-    private final int[] btreeFields;
+    protected final IBinaryComparatorFactory[] btreeComparatorFactories;
+    protected final IPrimitiveValueProviderFactory[] valueProviderFactories;
+    protected final RTreePolicyType rtreePolicyType;
+    protected final ILinearizeComparatorFactory linearizeCmpFactory;
+    protected final int[] btreeFields;
 
     public LSMRTreeDataflowHelperFactory(IPrimitiveValueProviderFactory[] valueProviderFactories,
             RTreePolicyType rtreePolicyType, IBinaryComparatorFactory[] btreeComparatorFactories,
diff --git a/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/AbstractLSMRTree.java b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/AbstractLSMRTree.java
index 2b9714d..a69f1f8 100644
--- a/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/AbstractLSMRTree.java
+++ b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/AbstractLSMRTree.java
@@ -65,8 +65,8 @@
     // For creating RTree's used in flush and merge.
     protected final ILSMComponentFactory componentFactory;
 
-    private IBinaryComparatorFactory[] btreeCmpFactories;
-    private IBinaryComparatorFactory[] rtreeCmpFactories;
+    protected IBinaryComparatorFactory[] btreeCmpFactories;
+    protected IBinaryComparatorFactory[] rtreeCmpFactories;
 
     // Common for in-memory and on-disk components.
     protected final ITreeIndexFrameFactory rtreeInteriorFrameFactory;
@@ -114,6 +114,31 @@
         this.comparatorFields = comparatorFields;
         this.linearizerArray = linearizerArray;
     }
+    
+    /*
+     * For External indexes with no memory components
+     */
+    public AbstractLSMRTree(ITreeIndexFrameFactory rtreeInteriorFrameFactory,
+            ITreeIndexFrameFactory rtreeLeafFrameFactory, ITreeIndexFrameFactory btreeInteriorFrameFactory,
+            ITreeIndexFrameFactory btreeLeafFrameFactory, ILSMIndexFileManager fileManager,
+            ILSMComponentFactory componentFactory, IFileMapProvider diskFileMapProvider, int fieldCount,
+            IBinaryComparatorFactory[] rtreeCmpFactories, IBinaryComparatorFactory[] btreeCmpFactories,
+            ILinearizeComparatorFactory linearizer, int[] comparatorFields, IBinaryComparatorFactory[] linearizerArray,
+            double bloomFilterFalsePositiveRate, ILSMMergePolicy mergePolicy, ILSMOperationTracker opTracker,
+            ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallback ioOpCallback) {
+        super(componentFactory.getBufferCache(), fileManager, diskFileMapProvider, bloomFilterFalsePositiveRate,
+                mergePolicy, opTracker, ioScheduler, ioOpCallback);
+        this.rtreeInteriorFrameFactory = rtreeInteriorFrameFactory;
+        this.rtreeLeafFrameFactory = rtreeLeafFrameFactory;
+        this.btreeInteriorFrameFactory = btreeInteriorFrameFactory;
+        this.btreeLeafFrameFactory = btreeLeafFrameFactory;
+        this.componentFactory = componentFactory;
+        this.btreeCmpFactories = btreeCmpFactories;
+        this.rtreeCmpFactories = rtreeCmpFactories;
+        this.linearizer = linearizer;
+        this.comparatorFields = comparatorFields;
+        this.linearizerArray = linearizerArray;
+    }
 
     @Override
     public synchronized void create() throws HyracksDataException {
diff --git a/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/ExternalRTree.java b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/ExternalRTree.java
new file mode 100644
index 0000000..d26d330
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/ExternalRTree.java
@@ -0,0 +1,758 @@
+/*
+ * 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.hyracks.storage.am.lsm.rtree.impls;
+
+import java.io.IOException;
+import java.util.LinkedList;
+import java.util.List;
+
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.ILinearizeComparatorFactory;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
+import edu.uci.ics.hyracks.storage.am.bloomfilter.impls.BloomCalculations;
+import edu.uci.ics.hyracks.storage.am.bloomfilter.impls.BloomFilter;
+import edu.uci.ics.hyracks.storage.am.bloomfilter.impls.BloomFilterFactory;
+import edu.uci.ics.hyracks.storage.am.bloomfilter.impls.BloomFilterSpecification;
+import edu.uci.ics.hyracks.storage.am.btree.impls.BTree;
+import edu.uci.ics.hyracks.storage.am.btree.impls.BTree.BTreeBulkLoader;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexBulkLoader;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexCursor;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexOperationContext;
+import edu.uci.ics.hyracks.storage.am.common.api.IModificationOperationCallback;
+import edu.uci.ics.hyracks.storage.am.common.api.ISearchOperationCallback;
+import edu.uci.ics.hyracks.storage.am.common.api.ISearchPredicate;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexCursor;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
+import edu.uci.ics.hyracks.storage.am.common.api.ITwoPCIndexBulkLoader;
+import edu.uci.ics.hyracks.storage.am.common.api.IndexException;
+import edu.uci.ics.hyracks.storage.am.common.api.TreeIndexException;
+import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallback;
+import edu.uci.ics.hyracks.storage.am.common.ophelpers.IndexOperation;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ITwoPCIndex;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMComponent;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndexAccessorInternal;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndexFileManager;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndexOperationContext;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMMergePolicy;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTracker;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.BlockingIOOperationCallbackWrapper;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.LSMOperationType;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.ExternalIndexHarness;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.LSMComponentFileReferences;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.TreeIndexFactory;
+import edu.uci.ics.hyracks.storage.am.rtree.impls.RTree;
+import edu.uci.ics.hyracks.storage.am.rtree.impls.SearchPredicate;
+import edu.uci.ics.hyracks.storage.common.file.IFileMapProvider;
+
+/**
+ * This is an lsm r-tree that does not have memory component and is modified
+ * only by bulk loading and addition of disk components as of this point, it is
+ * intended for use with external dataset indexes only.
+ * 
+ * @author alamouda
+ */
+public class ExternalRTree extends LSMRTree implements ITwoPCIndex {
+
+    // A second disk component list that will be used when a transaction is
+    // committed and will be seen by subsequent accessors
+    private List<ILSMComponent> secondDiskComponents;
+    // A pointer that points to the current most recent list (either
+    // diskComponents = 0, or secondDiskComponents = 1). It starts with -1 to
+    // indicate first time activation
+    private int version = -1;
+    private int fieldCount;
+
+    public ExternalRTree(ITreeIndexFrameFactory rtreeInteriorFrameFactory,
+            ITreeIndexFrameFactory rtreeLeafFrameFactory, ITreeIndexFrameFactory btreeInteriorFrameFactory,
+            ITreeIndexFrameFactory btreeLeafFrameFactory, ILSMIndexFileManager fileNameManager,
+            TreeIndexFactory<RTree> diskRTreeFactory, TreeIndexFactory<BTree> diskBTreeFactory,
+            BloomFilterFactory bloomFilterFactory, double bloomFilterFalsePositiveRate,
+            IFileMapProvider diskFileMapProvider, int fieldCount, IBinaryComparatorFactory[] rtreeCmpFactories,
+            IBinaryComparatorFactory[] btreeCmpFactories, ILinearizeComparatorFactory linearizer,
+            int[] comparatorFields, IBinaryComparatorFactory[] linearizerArray, ILSMMergePolicy mergePolicy,
+            ILSMOperationTracker opTracker, ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallback ioOpCallback,
+            int[] buddyBTreeFields, int version) {
+        super(rtreeInteriorFrameFactory, rtreeLeafFrameFactory, btreeInteriorFrameFactory, btreeLeafFrameFactory,
+                fileNameManager, diskRTreeFactory, diskBTreeFactory, bloomFilterFactory, bloomFilterFalsePositiveRate,
+                diskFileMapProvider, fieldCount, rtreeCmpFactories, btreeCmpFactories, linearizer, comparatorFields,
+                linearizerArray, mergePolicy, opTracker, ioScheduler, ioOpCallback, buddyBTreeFields);
+        this.secondDiskComponents = new LinkedList<ILSMComponent>();
+        this.version = version;
+        this.fieldCount = fieldCount;
+    }
+
+    // This method is used to create a target for a bulk modify operation. This
+    // component must then eventually be either committed or deleted
+    private ILSMComponent createTransactionTarget() throws HyracksDataException, IndexException {
+        LSMComponentFileReferences componentFileRefs;
+        try {
+            componentFileRefs = fileManager.getNewTransactionFileReference();
+        } catch (IOException e) {
+            throw new HyracksDataException("Failed to create transaction components", e);
+        }
+        return createDiskComponent(componentFactory, componentFileRefs.getInsertIndexFileReference(),
+                componentFileRefs.getDeleteIndexFileReference(), componentFileRefs.getBloomFilterFileReference(), true);
+    }
+
+    // The subsume merged components is overridden to account for:
+    // 1. the number of readers of components
+    // 2. maintaining two versions of the index
+    @Override
+    public void subsumeMergedComponents(ILSMComponent newComponent, List<ILSMComponent> mergedComponents)
+            throws HyracksDataException {
+        // determine which list is the new one
+        List<ILSMComponent> newerList;
+        List<ILSMComponent> olderList;
+        if (version == 0) {
+            newerList = diskComponents;
+            olderList = secondDiskComponents;
+        } else {
+            newerList = secondDiskComponents;
+            olderList = diskComponents;
+        }
+        // check if merge will affect the older list
+        if (olderList.containsAll(mergedComponents)) {
+            int swapIndex = olderList.indexOf(mergedComponents.get(0));
+            olderList.removeAll(mergedComponents);
+            olderList.add(swapIndex, newComponent);
+        }
+        // The new list will always have all the merged components
+        int swapIndex = newerList.indexOf(mergedComponents.get(0));
+        newerList.removeAll(mergedComponents);
+        newerList.add(swapIndex, newComponent);
+    }
+
+    // This method is used by the merge policy when it needs to check if a merge
+    // is needed.
+    // It only needs to return the newer list
+    @Override
+    public List<ILSMComponent> getImmutableComponents() {
+        if (version == 0) {
+            return diskComponents;
+        } else {
+            return secondDiskComponents;
+        }
+    }
+
+    // This function should only be used when a transaction fail. it doesn't
+    // take any parameters since there can only be
+    // a single transaction and hence a single transaction component on disk
+    public void deleteTransactionComponent() throws HyracksDataException {
+        fileManager.deleteTransactionFiles();
+    }
+
+    // This function in an instance of this index is only used after a bulk load
+    // is successful
+    // it will therefore add the component to the first list and enter it.
+    @Override
+    public void addComponent(ILSMComponent c) throws HyracksDataException {
+        if (version == 0) {
+            diskComponents.add(0, c);
+        } else if (version == 1) {
+            secondDiskComponents.add(0, c);
+        }
+    }
+
+    // This function is used when a new component is to be committed.
+    public void commitTransactionDiskComponent(ILSMComponent newComponent) throws HyracksDataException {
+
+        // determine which list is the new one and flip the pointer
+        List<ILSMComponent> newerList;
+        List<ILSMComponent> olderList;
+        if (version == 0) {
+            newerList = diskComponents;
+            olderList = secondDiskComponents;
+            version = 1;
+        } else {
+            newerList = secondDiskComponents;
+            olderList = diskComponents;
+            version = 0;
+        }
+        // Remove components from list
+        olderList.clear();
+        // Add components
+        olderList.addAll(newerList);
+        if (newComponent != null) {
+            // Add it to the list
+            olderList.add(0, newComponent);
+        }
+    }
+
+    @Override
+    public synchronized void activate() throws HyracksDataException {
+        if (isActivated) {
+            throw new HyracksDataException("Failed to activate the index since it is already activated.");
+        }
+
+        if (diskComponents.size() == 0 && secondDiskComponents.size() == 0) {
+            //First time activation
+            List<LSMComponentFileReferences> validFileReferences;
+            try {
+                validFileReferences = fileManager.cleanupAndGetValidFiles();
+            } catch (IndexException e) {
+                throw new HyracksDataException(e);
+            }
+            for (LSMComponentFileReferences lsmComonentFileReference : validFileReferences) {
+                LSMRTreeDiskComponent component;
+                try {
+                    component = createDiskComponent(componentFactory,
+                            lsmComonentFileReference.getInsertIndexFileReference(),
+                            lsmComonentFileReference.getDeleteIndexFileReference(),
+                            lsmComonentFileReference.getBloomFilterFileReference(), false);
+                } catch (IndexException e) {
+                    throw new HyracksDataException(e);
+                }
+                diskComponents.add(component);
+                secondDiskComponents.add(component);
+            }
+            ((ExternalIndexHarness) lsmHarness).indexFirstTimeActivated();
+        } else {
+            // This index has been opened before or is brand new with no components
+            // components. It should also maintain the version pointer
+            for (ILSMComponent c : diskComponents) {
+                LSMRTreeDiskComponent component = (LSMRTreeDiskComponent) c;
+                RTree rtree = component.getRTree();
+                BTree btree = component.getBTree();
+                BloomFilter bloomFilter = component.getBloomFilter();
+                rtree.activate();
+                btree.activate();
+                bloomFilter.activate();
+            }
+            for (ILSMComponent c : secondDiskComponents) {
+                // Only activate non shared components
+                if (!diskComponents.contains(c)) {
+                    LSMRTreeDiskComponent component = (LSMRTreeDiskComponent) c;
+                    RTree rtree = component.getRTree();
+                    BTree btree = component.getBTree();
+                    BloomFilter bloomFilter = component.getBloomFilter();
+                    rtree.activate();
+                    btree.activate();
+                    bloomFilter.activate();
+                }
+            }
+        }
+        isActivated = true;
+    }
+
+    @Override
+    public synchronized void create() throws HyracksDataException {
+        super.create();
+        secondDiskComponents.clear();
+    }
+
+    // we override this method because this index uses a different opcontext
+    @Override
+    public void search(ILSMIndexOperationContext ictx, IIndexCursor cursor, ISearchPredicate pred)
+            throws HyracksDataException, IndexException {
+        ExternalRTreeOpContext ctx = (ExternalRTreeOpContext) ictx;
+        List<ILSMComponent> operationalComponents = ictx.getComponentHolder();
+
+        LSMRTreeCursorInitialState initialState = new LSMRTreeCursorInitialState(rtreeLeafFrameFactory,
+                rtreeInteriorFrameFactory, btreeLeafFrameFactory, ctx.getBTreeMultiComparator(), lsmHarness,
+                comparatorFields, linearizerArray, ctx.searchCallback, operationalComponents);
+
+        cursor.open(initialState, pred);
+    }
+
+    // The only reason for overriding the merge method is the way to determine
+    // the need to keep deleted tuples
+    // This can be done in a better way by creating a method boolean
+    // keepDeletedTuples(mergedComponents);
+    @Override
+    public ILSMComponent merge(ILSMIOOperation operation) throws HyracksDataException, IndexException {
+        LSMRTreeMergeOperation mergeOp = (LSMRTreeMergeOperation) operation;
+        ITreeIndexCursor cursor = mergeOp.getCursor();
+        ISearchPredicate rtreeSearchPred = new SearchPredicate(null, null);
+        ILSMIndexOperationContext opCtx = ((LSMRTreeSortedCursor) cursor).getOpCtx();
+        opCtx.getComponentHolder().addAll(mergeOp.getMergingComponents());
+        search(opCtx, cursor, rtreeSearchPred);
+
+        LSMRTreeDiskComponent mergedComponent = createDiskComponent(componentFactory, mergeOp.getRTreeMergeTarget(),
+                mergeOp.getBTreeMergeTarget(), mergeOp.getBloomFilterMergeTarget(), true);
+
+        // In case we must keep the deleted-keys BTrees, then they must be
+        // merged *before* merging the r-trees so that
+        // lsmHarness.endSearch() is called once when the r-trees have been
+        // merged.
+        if (mergeOp.isKeepDeletedTuples()) {
+            // Keep the deleted tuples since the oldest disk component is not
+            // included in the merge operation
+
+            LSMRTreeDeletedKeysBTreeMergeCursor btreeCursor = new LSMRTreeDeletedKeysBTreeMergeCursor(opCtx);
+            search(opCtx, btreeCursor, rtreeSearchPred);
+
+            BTree btree = mergedComponent.getBTree();
+            IIndexBulkLoader btreeBulkLoader = btree.createBulkLoader(1.0f, true, 0L, false);
+
+            long numElements = 0L;
+            for (int i = 0; i < mergeOp.getMergingComponents().size(); ++i) {
+                numElements += ((LSMRTreeDiskComponent) mergeOp.getMergingComponents().get(i)).getBloomFilter()
+                        .getNumElements();
+            }
+
+            int maxBucketsPerElement = BloomCalculations.maxBucketsPerElement(numElements);
+            BloomFilterSpecification bloomFilterSpec = BloomCalculations.computeBloomSpec(maxBucketsPerElement,
+                    bloomFilterFalsePositiveRate);
+            IIndexBulkLoader builder = mergedComponent.getBloomFilter().createBuilder(numElements,
+                    bloomFilterSpec.getNumHashes(), bloomFilterSpec.getNumBucketsPerElements());
+
+            try {
+                while (btreeCursor.hasNext()) {
+                    btreeCursor.next();
+                    ITupleReference tuple = btreeCursor.getTuple();
+                    btreeBulkLoader.add(tuple);
+                    builder.add(tuple);
+                }
+            } finally {
+                btreeCursor.close();
+                builder.end();
+            }
+            btreeBulkLoader.end();
+        }
+
+        IIndexBulkLoader bulkLoader = mergedComponent.getRTree().createBulkLoader(1.0f, false, 0L, false);
+        try {
+            while (cursor.hasNext()) {
+                cursor.next();
+                ITupleReference frameTuple = cursor.getTuple();
+                bulkLoader.add(frameTuple);
+            }
+        } finally {
+            cursor.close();
+        }
+        bulkLoader.end();
+        return mergedComponent;
+    }
+
+    @Override
+    public void deactivate(boolean flushOnExit) throws HyracksDataException {
+        if (!isActivated) {
+            throw new HyracksDataException("Failed to deactivate the index since it is already deactivated.");
+        }
+
+        if (flushOnExit) {
+            BlockingIOOperationCallbackWrapper cb = new BlockingIOOperationCallbackWrapper(ioOpCallback);
+            cb.afterFinalize(LSMOperationType.FLUSH, null);
+        }
+
+        for (ILSMComponent c : diskComponents) {
+            LSMRTreeDiskComponent component = (LSMRTreeDiskComponent) c;
+            RTree rtree = component.getRTree();
+            BTree btree = component.getBTree();
+            BloomFilter bloomFilter = component.getBloomFilter();
+            rtree.deactivate();
+            btree.deactivate();
+            bloomFilter.deactivate();
+        }
+        for (ILSMComponent c : secondDiskComponents) {
+            // Only deactivate non shared components
+            if (!diskComponents.contains(c)) {
+                LSMRTreeDiskComponent component = (LSMRTreeDiskComponent) c;
+                RTree rtree = component.getRTree();
+                BTree btree = component.getBTree();
+                BloomFilter bloomFilter = component.getBloomFilter();
+                rtree.deactivate();
+                btree.deactivate();
+                bloomFilter.deactivate();
+            }
+        }
+        isActivated = false;
+    }
+
+    // The clear method is not used anywhere in AsterixDB! we override it anyway
+    // to exit components first and clear the two lists
+    @Override
+    public void clear() throws HyracksDataException {
+        if (!isActivated) {
+            throw new HyracksDataException("Failed to clear the index since it is not activated.");
+        }
+        ((ExternalIndexHarness) lsmHarness).indexClear();
+
+        for (ILSMComponent c : diskComponents) {
+            LSMRTreeDiskComponent component = (LSMRTreeDiskComponent) c;
+            component.getRTree().deactivate();
+            component.getBloomFilter().deactivate();
+            component.getBTree().deactivate();
+            component.getRTree().destroy();
+            component.getBloomFilter().destroy();
+            component.getBTree().destroy();
+            // Remove from second list to avoid destroying twice
+            secondDiskComponents.remove(c);
+        }
+
+        for (ILSMComponent c : secondDiskComponents) {
+            LSMRTreeDiskComponent component = (LSMRTreeDiskComponent) c;
+            component.getRTree().deactivate();
+            component.getBloomFilter().deactivate();
+            component.getBTree().deactivate();
+            component.getRTree().destroy();
+            component.getBloomFilter().destroy();
+            component.getBTree().destroy();
+        }
+
+        diskComponents.clear();
+        secondDiskComponents.clear();
+        version = -1;
+    }
+
+    @Override
+    public void destroy() throws HyracksDataException {
+        if (isActivated) {
+            throw new HyracksDataException("Failed to destroy the index since it is activated.");
+        }
+        for (ILSMComponent c : diskComponents) {
+            LSMRTreeDiskComponent component = (LSMRTreeDiskComponent) c;
+            component.getRTree().destroy();
+            component.getBTree().destroy();
+            component.getBloomFilter().destroy();
+            // Remove from second list to avoid destroying twice
+            secondDiskComponents.remove(c);
+        }
+        for (ILSMComponent c : secondDiskComponents) {
+            LSMRTreeDiskComponent component = (LSMRTreeDiskComponent) c;
+            component.getRTree().destroy();
+            component.getBTree().destroy();
+            component.getBloomFilter().destroy();
+        }
+        diskComponents.clear();
+        secondDiskComponents.clear();
+        fileManager.deleteDirs();
+        version = -1;
+    }
+
+    // Not supported
+    @Override
+    public void modify(IIndexOperationContext ictx, ITupleReference tuple) throws HyracksDataException, IndexException {
+        throw new UnsupportedOperationException("tuple modify not supported in LSM-Disk-Only-RTree");
+    }
+
+    // Not supported
+    @Override
+    public void scheduleFlush(ILSMIndexOperationContext ctx, ILSMIOOperationCallback callback)
+            throws HyracksDataException {
+        throw new UnsupportedOperationException("flush not supported in LSM-Disk-Only-RTree");
+    }
+
+    // Not supported
+    @Override
+    public ILSMComponent flush(ILSMIOOperation operation) throws HyracksDataException, IndexException {
+        throw new UnsupportedOperationException("flush not supported in LSM-Disk-Only-RTree");
+    }
+
+    // Only support search and merge operations
+    @Override
+    public void getOperationalComponents(ILSMIndexOperationContext ctx) {
+        List<ILSMComponent> operationalComponents = ctx.getComponentHolder();
+        List<ILSMComponent> immutableComponents;
+        // Identify current list in case of a merge
+        if (version == 0) {
+            immutableComponents = diskComponents;
+        } else {
+            immutableComponents = secondDiskComponents;
+        }
+        ExternalRTreeOpContext opCtx = (ExternalRTreeOpContext) ctx;
+        operationalComponents.clear();
+        switch (ctx.getOperation()) {
+            case SEARCH:
+                if (opCtx.getTargetIndexVersion() == 0) {
+                    operationalComponents.addAll(diskComponents);
+                } else {
+                    operationalComponents.addAll(secondDiskComponents);
+                }
+                break;
+            case MERGE:
+                operationalComponents.addAll(ctx.getComponentsToBeMerged());
+                break;
+            case FULL_MERGE:
+                operationalComponents.addAll(immutableComponents);
+                break;
+            case FLUSH:
+                // Do nothing. this is left here even though the index never
+                // performs flushes because a flush is triggered by
+                // dataset lifecycle manager when closing an index. Having no
+                // components is a no operation
+                break;
+            default:
+                throw new UnsupportedOperationException("Operation " + ctx.getOperation() + " not supported.");
+        }
+    }
+
+    // For initial load
+    @Override
+    public IIndexBulkLoader createBulkLoader(float fillLevel, boolean verifyInput, long numElementsHint,
+            boolean checkIfEmptyIndex) throws TreeIndexException {
+        try {
+            return new LSMTwoPCRTreeBulkLoader(fillLevel, verifyInput, 0, checkIfEmptyIndex, false);
+        } catch (HyracksDataException e) {
+            throw new TreeIndexException(e);
+        }
+    }
+
+    // For transaction bulk load <- could consolidate with the above method ->
+    @Override
+    public IIndexBulkLoader createTransactionBulkLoader(float fillLevel, boolean verifyInput, long numElementsHint,
+            boolean checkIfEmptyIndex) throws TreeIndexException {
+        try {
+            return new LSMTwoPCRTreeBulkLoader(fillLevel, verifyInput, numElementsHint, checkIfEmptyIndex, true);
+        } catch (HyracksDataException e) {
+            throw new TreeIndexException(e);
+        }
+    }
+
+    // The bulk loader used for both initial loading and transaction
+    // modifications
+    public class LSMTwoPCRTreeBulkLoader implements IIndexBulkLoader, ITwoPCIndexBulkLoader {
+        private final ILSMComponent component;
+        private final IIndexBulkLoader rtreeBulkLoader;
+        private final BTreeBulkLoader btreeBulkLoader;
+        private final IIndexBulkLoader builder;
+        private boolean cleanedUpArtifacts = false;
+        private boolean isEmptyComponent = true;
+        private boolean endedBloomFilterLoad = false;
+        private boolean isTransaction;
+
+        public LSMTwoPCRTreeBulkLoader(float fillFactor, boolean verifyInput, long numElementsHint,
+                boolean checkIfEmptyIndex, boolean isTransaction) throws TreeIndexException, HyracksDataException {
+            this.isTransaction = isTransaction;
+            // Create the appropriate target
+            if (isTransaction) {
+                try {
+                    component = createTransactionTarget();
+                } catch (HyracksDataException | IndexException e) {
+                    throw new TreeIndexException(e);
+                }
+            } else {
+                if (checkIfEmptyIndex && !isEmptyIndex()) {
+                    throw new TreeIndexException("Cannot load an index that is not empty");
+                }
+                try {
+                    component = createBulkLoadTarget();
+                } catch (HyracksDataException | IndexException e) {
+                    throw new TreeIndexException(e);
+                }
+            }
+
+            // Create the three loaders
+            rtreeBulkLoader = ((LSMRTreeDiskComponent) component).getRTree().createBulkLoader(fillFactor, verifyInput,
+                    numElementsHint, false);
+            btreeBulkLoader = (BTreeBulkLoader) ((LSMRTreeDiskComponent) component).getBTree().createBulkLoader(
+                    fillFactor, verifyInput, numElementsHint, false);
+            int maxBucketsPerElement = BloomCalculations.maxBucketsPerElement(numElementsHint);
+            BloomFilterSpecification bloomFilterSpec = BloomCalculations.computeBloomSpec(maxBucketsPerElement,
+                    bloomFilterFalsePositiveRate);
+            builder = ((LSMRTreeDiskComponent) component).getBloomFilter().createBuilder(numElementsHint,
+                    bloomFilterSpec.getNumHashes(), bloomFilterSpec.getNumBucketsPerElements());
+        }
+
+        @Override
+        public void add(ITupleReference tuple) throws IndexException, HyracksDataException {
+            try {
+                rtreeBulkLoader.add(tuple);
+            } catch (IndexException | HyracksDataException | RuntimeException e) {
+                cleanupArtifacts();
+                throw e;
+            }
+            if (isEmptyComponent) {
+                isEmptyComponent = false;
+            }
+        }
+
+        // This is made public in case of a failure, it is better to delete all
+        // created artifacts.
+        public void cleanupArtifacts() throws HyracksDataException {
+            if (!cleanedUpArtifacts) {
+                cleanedUpArtifacts = true;
+                try {
+                    ((LSMRTreeDiskComponent) component).getRTree().deactivate();
+                } catch (Exception e) {
+
+                }
+                ((LSMRTreeDiskComponent) component).getRTree().destroy();
+                try {
+                    ((LSMRTreeDiskComponent) component).getBTree().deactivate();
+                } catch (Exception e) {
+
+                }
+                ((LSMRTreeDiskComponent) component).getBTree().destroy();
+                try {
+                    ((LSMRTreeDiskComponent) component).getBloomFilter().deactivate();
+                } catch (Exception e) {
+
+                }
+                ((LSMRTreeDiskComponent) component).getBloomFilter().destroy();
+            }
+        }
+
+        @Override
+        public void end() throws HyracksDataException, IndexException {
+            if (!cleanedUpArtifacts) {
+                if (!endedBloomFilterLoad) {
+                    builder.end();
+                    endedBloomFilterLoad = true;
+                }
+                rtreeBulkLoader.end();
+                btreeBulkLoader.end();
+                if (isEmptyComponent) {
+                    cleanupArtifacts();
+                } else if (isTransaction) {
+                    // Since this is a transaction component, validate and
+                    // deactivate. it could later be added or deleted
+                    markAsValid(component);
+                    RTree rtree = ((LSMRTreeDiskComponent) component).getRTree();
+                    BTree btree = ((LSMRTreeDiskComponent) component).getBTree();
+                    BloomFilter bloomFilter = ((LSMRTreeDiskComponent) component).getBloomFilter();
+                    rtree.deactivate();
+                    btree.deactivate();
+                    bloomFilter.deactivate();
+                } else {
+                    lsmHarness.addBulkLoadedComponent(component);
+                }
+            }
+        }
+
+        @Override
+        public void delete(ITupleReference tuple) throws IndexException, HyracksDataException {
+            try {
+                btreeBulkLoader.add(tuple);
+                builder.add(tuple);
+            } catch (IndexException | HyracksDataException | RuntimeException e) {
+                cleanupArtifacts();
+                throw e;
+            }
+            if (isEmptyComponent) {
+                isEmptyComponent = false;
+            }
+        }
+
+        @Override
+        public void abort() {
+            try {
+                cleanupArtifacts();
+            } catch (Exception e) {
+
+            }
+        }
+    }
+
+    @Override
+    public String toString() {
+        return "LSMTwoPCRTree [" + fileManager.getBaseDir() + "]";
+    }
+
+    // The only change the the schedule merge is the method used to create the
+    // opCtx. first line <- in schedule merge, we->
+    @Override
+    public void scheduleMerge(ILSMIndexOperationContext ctx, ILSMIOOperationCallback callback)
+            throws HyracksDataException, IndexException {
+        ILSMIndexOperationContext rctx = createOpContext(NoOpOperationCallback.INSTANCE, -1);
+        rctx.setOperation(IndexOperation.MERGE);
+        List<ILSMComponent> mergingComponents = ctx.getComponentHolder();
+        ITreeIndexCursor cursor = new LSMRTreeSortedCursor(rctx, linearizer, buddyBTreeFields);
+        LSMComponentFileReferences relMergeFileRefs = getMergeTargetFileName(mergingComponents);
+        ILSMIndexAccessorInternal accessor = new LSMRTreeAccessor(lsmHarness, rctx);
+        // create the merge operation.
+        LSMRTreeMergeOperation mergeOp = new LSMRTreeMergeOperation((ILSMIndexAccessorInternal) accessor,
+                mergingComponents, cursor, relMergeFileRefs.getInsertIndexFileReference(),
+                relMergeFileRefs.getDeleteIndexFileReference(), relMergeFileRefs.getBloomFilterFileReference(),
+                callback, fileManager.getBaseDir());
+        // set the keepDeletedTuples flag
+        boolean keepDeleteTuples = false;
+        if (version == 0) {
+            keepDeleteTuples = mergeOp.getMergingComponents().get(mergeOp.getMergingComponents().size() - 1) != diskComponents
+                    .get(diskComponents.size() - 1);
+        } else {
+            keepDeleteTuples = mergeOp.getMergingComponents().get(mergeOp.getMergingComponents().size() - 1) != secondDiskComponents
+                    .get(secondDiskComponents.size() - 1);
+        }
+        mergeOp.setKeepDeletedTuples(keepDeleteTuples);
+
+        ioScheduler.scheduleOperation(mergeOp);
+    }
+
+    @Override
+    public ILSMIndexAccessorInternal createAccessor(ISearchOperationCallback searchCallback, int targetIndexVersion)
+            throws HyracksDataException {
+        return new LSMRTreeAccessor(lsmHarness, createOpContext(searchCallback, targetIndexVersion));
+    }
+
+    // This method creates the appropriate opContext for the targeted version
+    public ExternalRTreeOpContext createOpContext(ISearchOperationCallback searchCallback, int targetVersion) {
+        return new ExternalRTreeOpContext(rtreeCmpFactories, btreeCmpFactories, searchCallback, targetVersion);
+    }
+
+    // The accessor for disk only indexes don't use modification callback and
+    // always carry the target index version with them
+    @Override
+    public ILSMIndexAccessorInternal createAccessor(IModificationOperationCallback modificationCallback,
+            ISearchOperationCallback searchCallback) {
+        return new LSMRTreeAccessor(lsmHarness, createOpContext(searchCallback, version));
+    }
+
+    public int getCurrentVersion() {
+        return version;
+    }
+
+    public List<ILSMComponent> getFirstComponentList() {
+        return diskComponents;
+    }
+
+    public List<ILSMComponent> getSecondComponentList() {
+        return secondDiskComponents;
+    }
+
+    @Override
+    public void commitTransaction() throws TreeIndexException, HyracksDataException, IndexException {
+        LSMComponentFileReferences componentFileRefrences = fileManager.getTransactionFileReferenceForCommit();
+        LSMRTreeDiskComponent component = null;
+        if (componentFileRefrences != null) {
+            component = createDiskComponent(componentFactory, componentFileRefrences.getInsertIndexFileReference(),
+                    componentFileRefrences.getDeleteIndexFileReference(),
+                    componentFileRefrences.getBloomFilterFileReference(), false);
+        }
+        ((ExternalIndexHarness) lsmHarness).addTransactionComponents(component);
+    }
+
+    @Override
+    public void abortTransaction() throws TreeIndexException {
+        try {
+            fileManager.deleteTransactionFiles();
+        } catch (HyracksDataException e) {
+            throw new TreeIndexException(e);
+        }
+    }
+
+    @Override
+    public void recoverTransaction() throws TreeIndexException {
+        try {
+            fileManager.recoverTransaction();
+        } catch (HyracksDataException e) {
+            throw new TreeIndexException(e);
+        }
+    }
+    
+    @Override
+    public boolean hasMemoryComponents(){
+        return false;
+    }
+    
+    @Override
+    public int getFieldCount() {
+        return fieldCount;
+    }
+}
diff --git a/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/ExternalRTreeOpContext.java b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/ExternalRTreeOpContext.java
new file mode 100644
index 0000000..55d0cad
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/ExternalRTreeOpContext.java
@@ -0,0 +1,103 @@
+/*
+ * 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.hyracks.storage.am.lsm.rtree.impls;
+
+import java.util.LinkedList;
+import java.util.List;
+
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import edu.uci.ics.hyracks.storage.am.common.api.IModificationOperationCallback;
+import edu.uci.ics.hyracks.storage.am.common.api.ISearchOperationCallback;
+import edu.uci.ics.hyracks.storage.am.common.ophelpers.IndexOperation;
+import edu.uci.ics.hyracks.storage.am.common.ophelpers.MultiComparator;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMComponent;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndexOperationContext;
+
+public class ExternalRTreeOpContext implements ILSMIndexOperationContext {
+	private IndexOperation op;
+	private MultiComparator bTreeCmp;
+	private MultiComparator rTreeCmp;
+	public final List<ILSMComponent> componentHolder;
+	private final List<ILSMComponent> componentsToBeMerged;
+	public final ISearchOperationCallback searchCallback;
+	private final int targetIndexVersion;
+
+	public ExternalRTreeOpContext(
+			IBinaryComparatorFactory[] rtreeCmpFactories,
+			IBinaryComparatorFactory[] btreeCmpFactories,
+			ISearchOperationCallback searchCallback, int targetIndexVersion) {
+
+		this.componentHolder = new LinkedList<ILSMComponent>();
+		this.componentsToBeMerged = new LinkedList<ILSMComponent>();
+		this.searchCallback = searchCallback;
+		this.targetIndexVersion = targetIndexVersion;
+		this.bTreeCmp = MultiComparator.create(btreeCmpFactories);
+		this.rTreeCmp = MultiComparator.create(rtreeCmpFactories);
+	}
+
+	public void setOperation(IndexOperation newOp) {
+		reset();
+		this.op = newOp;
+	}
+
+	@Override
+	public void setCurrentMutableComponentId(int currentMutableComponentId) {
+		// Do nothing. this should never be called for disk only indexes
+	}
+
+	@Override
+	public void reset() {
+		componentHolder.clear();
+		componentsToBeMerged.clear();
+	}
+
+	@Override
+	public IndexOperation getOperation() {
+		return op;
+	}
+
+	public MultiComparator getBTreeMultiComparator() {
+		return bTreeCmp;
+	}
+
+	public MultiComparator getRTreeMultiComparator() {
+		return rTreeCmp;
+	}
+
+	@Override
+	public List<ILSMComponent> getComponentHolder() {
+		return componentHolder;
+	}
+
+	@Override
+	public ISearchOperationCallback getSearchOperationCallback() {
+		return searchCallback;
+	}
+
+	// This should never be needed for disk only indexes
+	@Override
+	public IModificationOperationCallback getModificationCallback() {
+		return null;
+	}
+
+	@Override
+	public List<ILSMComponent> getComponentsToBeMerged() {
+		return componentsToBeMerged;
+	}
+
+	public int getTargetIndexVersion() {
+		return targetIndexVersion;
+	}
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTree.java b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTree.java
index 0d35ab3..37ce427 100644
--- a/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTree.java
+++ b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTree.java
@@ -66,7 +66,7 @@
 import edu.uci.ics.hyracks.storage.common.file.IFileMapProvider;
 
 public class LSMRTree extends AbstractLSMRTree {
-    private int[] buddyBTreeFields;
+    protected int[] buddyBTreeFields;
 
     public LSMRTree(List<IVirtualBufferCache> virtualBufferCaches, ITreeIndexFrameFactory rtreeInteriorFrameFactory,
             ITreeIndexFrameFactory rtreeLeafFrameFactory, ITreeIndexFrameFactory btreeInteriorFrameFactory,
@@ -85,6 +85,26 @@
                 mergePolicy, opTracker, ioScheduler, ioOpCallback);
         this.buddyBTreeFields = buddyBTreeFields;
     }
+    
+    /*
+     * For External indexes with no memory components
+     */
+    public LSMRTree(ITreeIndexFrameFactory rtreeInteriorFrameFactory, ITreeIndexFrameFactory rtreeLeafFrameFactory,
+            ITreeIndexFrameFactory btreeInteriorFrameFactory, ITreeIndexFrameFactory btreeLeafFrameFactory,
+            ILSMIndexFileManager fileNameManager, TreeIndexFactory<RTree> diskRTreeFactory,
+            TreeIndexFactory<BTree> diskBTreeFactory, BloomFilterFactory bloomFilterFactory,
+            double bloomFilterFalsePositiveRate, IFileMapProvider diskFileMapProvider, int fieldCount,
+            IBinaryComparatorFactory[] rtreeCmpFactories, IBinaryComparatorFactory[] btreeCmpFactories,
+            ILinearizeComparatorFactory linearizer, int[] comparatorFields, IBinaryComparatorFactory[] linearizerArray,
+            ILSMMergePolicy mergePolicy, ILSMOperationTracker opTracker, ILSMIOOperationScheduler ioScheduler,
+            ILSMIOOperationCallback ioOpCallback, int[] buddyBTreeFields) {
+        super(rtreeInteriorFrameFactory, rtreeLeafFrameFactory, btreeInteriorFrameFactory, btreeLeafFrameFactory,
+                fileNameManager, new LSMRTreeDiskComponentFactory(diskRTreeFactory, diskBTreeFactory,
+                        bloomFilterFactory), diskFileMapProvider, fieldCount, rtreeCmpFactories, btreeCmpFactories,
+                linearizer, comparatorFields, linearizerArray, bloomFilterFalsePositiveRate, mergePolicy, opTracker,
+                ioScheduler, ioOpCallback);
+        this.buddyBTreeFields = buddyBTreeFields;
+    }
 
     /**
      * Opens LSMRTree, cleaning up invalid files from base dir, and registering
@@ -405,7 +425,7 @@
         }
     }
 
-    private ILSMComponent createBulkLoadTarget() throws HyracksDataException, IndexException {
+    protected ILSMComponent createBulkLoadTarget() throws HyracksDataException, IndexException {
         LSMComponentFileReferences componentFileRefs = fileManager.getRelFlushFileReference();
         return createDiskComponent(componentFactory, componentFileRefs.getInsertIndexFileReference(),
                 componentFileRefs.getDeleteIndexFileReference(), componentFileRefs.getBloomFilterFileReference(), true);
diff --git a/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTreeFileManager.java b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTreeFileManager.java
index 9502d5e..cca640f 100644
--- a/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTreeFileManager.java
+++ b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTreeFileManager.java
@@ -17,6 +17,9 @@
 
 import java.io.File;
 import java.io.FilenameFilter;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.nio.file.Paths;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.Date;
@@ -77,7 +80,8 @@
         String[] lastTimestampRange = lastFileName.split(SPLIT_STRING);
 
         String baseName = baseDir + firstTimestampRange[0] + SPLIT_STRING + lastTimestampRange[1];
-        // Get the range of timestamps by taking the earliest and the latest timestamps
+        // Get the range of timestamps by taking the earliest and the latest
+        // timestamps
         return new LSMComponentFileReferences(createMergeFile(baseName + SPLIT_STRING + RTREE_STRING),
                 createMergeFile(baseName + SPLIT_STRING + BTREE_STRING), createMergeFile(baseName + SPLIT_STRING
                         + BLOOM_FILTER_STRING));
@@ -90,15 +94,18 @@
         ArrayList<ComparableFileName> allBTreeFiles = new ArrayList<ComparableFileName>();
         ArrayList<ComparableFileName> allBloomFilterFiles = new ArrayList<ComparableFileName>();
 
+        // Create a transaction filter <- to hide transaction components->
+        FilenameFilter transactionFilter = getTransactionFileFilter(false);
+
         // Gather files.
-        cleanupAndGetValidFilesInternal(btreeFilter, btreeFactory, allBTreeFiles);
+        cleanupAndGetValidFilesInternal(getCompoundFilter(transactionFilter, btreeFilter), btreeFactory, allBTreeFiles);
         HashSet<String> btreeFilesSet = new HashSet<String>();
         for (ComparableFileName cmpFileName : allBTreeFiles) {
             int index = cmpFileName.fileName.lastIndexOf(SPLIT_STRING);
             btreeFilesSet.add(cmpFileName.fileName.substring(0, index));
         }
-        validateFiles(btreeFilesSet, allRTreeFiles, rtreeFilter, rtreeFactory);
-        validateFiles(btreeFilesSet, allBloomFilterFiles, bloomFilterFilter, null);
+        validateFiles(btreeFilesSet, allRTreeFiles, getCompoundFilter(transactionFilter, rtreeFilter), rtreeFactory);
+        validateFiles(btreeFilesSet, allBloomFilterFiles, getCompoundFilter(transactionFilter, bloomFilterFilter), null);
 
         // Sanity check.
         if (allRTreeFiles.size() != allBTreeFiles.size() || allBTreeFiles.size() != allBloomFilterFiles.size()) {
@@ -183,7 +190,62 @@
             validFiles.add(new LSMComponentFileReferences(cmpRTreeFileName.fileRef, cmpBTreeFileName.fileRef,
                     cmpBloomFilterFileName.fileRef));
         }
-
         return validFiles;
     }
+
+    @Override
+    public LSMComponentFileReferences getNewTransactionFileReference() throws IOException {
+        Date date = new Date();
+        String ts = formatter.format(date);
+        // Create transaction lock file
+        Files.createFile(Paths.get(baseDir + TRANSACTION_PREFIX + ts));
+
+        String baseName = baseDir + ts + SPLIT_STRING + ts;
+        return new LSMComponentFileReferences(createFlushFile(baseName + SPLIT_STRING + RTREE_STRING),
+                createFlushFile(baseName + SPLIT_STRING + BTREE_STRING), createFlushFile(baseName + SPLIT_STRING
+                        + BLOOM_FILTER_STRING));
+    }
+
+    @Override
+    public LSMComponentFileReferences getTransactionFileReferenceForCommit() throws HyracksDataException {
+        FilenameFilter transactionFilter;
+        File dir = new File(baseDir);
+        String[] files = dir.list(transactionFileNameFilter);
+        if (files.length == 0)
+            return null;
+        if (files.length != 1) {
+            throw new HyracksDataException("More than one transaction lock found:" + files.length);
+        } else {
+            transactionFilter = getTransactionFileFilter(true);
+            String txnFileName = dir.getPath() + File.separator + files[0];
+            // get the actual transaction files
+            files = dir.list(transactionFilter);
+            if (files.length < 3) {
+                throw new HyracksDataException("LSM Rtree transaction has less than 3 files :" + files.length);
+            }
+            try {
+                Files.delete(Paths.get(txnFileName));
+            } catch (IOException e) {
+                throw new HyracksDataException("Failed to delete transaction lock :" + txnFileName);
+            }
+        }
+        File rTreeFile = null;
+        File bTreeFile = null;
+        File bloomFilterFile = null;
+        for (String fileName : files) {
+            if (fileName.endsWith(BTREE_STRING)) {
+                bTreeFile = new File(dir.getPath() + File.separator + fileName);
+            } else if (fileName.endsWith(RTREE_STRING)) {
+                rTreeFile = new File(dir.getPath() + File.separator + fileName);
+            } else if (fileName.endsWith(BLOOM_FILTER_STRING)) {
+                bloomFilterFile = new File(dir.getPath() + File.separator + fileName);
+            } else {
+                throw new HyracksDataException("unrecognized file found = " + fileName);
+            }
+        }
+        FileReference rTreeFileRef = new FileReference(rTreeFile);
+        FileReference bTreeFileRef = new FileReference(bTreeFile);
+        FileReference bloomFilterFileRef = new FileReference(bloomFilterFile);
+        return new LSMComponentFileReferences(rTreeFileRef, bTreeFileRef, bloomFilterFileRef);
+    }
 }
diff --git a/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTreeMergeOperation.java b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTreeMergeOperation.java
index ddf3ebf..4b94600 100644
--- a/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTreeMergeOperation.java
+++ b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTreeMergeOperation.java
@@ -37,6 +37,7 @@
     private final FileReference bloomFilterMergeTarget;
     private final ILSMIOOperationCallback callback;
     private final String indexIdentifier;
+    private boolean keepDeletedTuples;
 
     public LSMRTreeMergeOperation(ILSMIndexAccessorInternal accessor, List<ILSMComponent> mergingComponents,
             ITreeIndexCursor cursor, FileReference rtreeMergeTarget, FileReference btreeMergeTarget,
@@ -116,4 +117,12 @@
     public LSMIOOpertionType getIOOpertionType() {
         return LSMIOOpertionType.MERGE;
     }
+
+    public boolean isKeepDeletedTuples() {
+        return keepDeletedTuples;
+    }
+
+    public void setKeepDeletedTuples(boolean keepDeletedTuples) {
+        this.keepDeletedTuples = keepDeletedTuples;
+    }
 }
diff --git a/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/utils/LSMRTreeUtils.java b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/utils/LSMRTreeUtils.java
index 13ecc84..224e449 100644
--- a/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/utils/LSMRTreeUtils.java
+++ b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/utils/LSMRTreeUtils.java
@@ -41,6 +41,7 @@
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.IVirtualBufferCache;
 import edu.uci.ics.hyracks.storage.am.lsm.common.impls.BTreeFactory;
 import edu.uci.ics.hyracks.storage.am.lsm.common.impls.TreeIndexFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.rtree.impls.ExternalRTree;
 import edu.uci.ics.hyracks.storage.am.lsm.rtree.impls.LSMRTree;
 import edu.uci.ics.hyracks.storage.am.lsm.rtree.impls.LSMRTreeFileManager;
 import edu.uci.ics.hyracks.storage.am.lsm.rtree.impls.LSMRTreeWithAntiMatterTuples;
@@ -168,6 +169,62 @@
         return lsmTree;
     }
 
+    public static ExternalRTree createExternalRTree(FileReference file, IBufferCache diskBufferCache,
+            IFileMapProvider diskFileMapProvider, ITypeTraits[] typeTraits,
+            IBinaryComparatorFactory[] rtreeCmpFactories, IBinaryComparatorFactory[] btreeCmpFactories,
+            IPrimitiveValueProviderFactory[] valueProviderFactories, RTreePolicyType rtreePolicyType,
+            double bloomFilterFalsePositiveRate, ILSMMergePolicy mergePolicy, ILSMOperationTracker opTracker,
+            ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallback ioOpCallback,
+            ILinearizeComparatorFactory linearizeCmpFactory, int[] buddyBTreeFields, int startWithVersion)
+            throws TreeIndexException {
+
+        ITypeTraits[] btreeTypeTraits = new ITypeTraits[buddyBTreeFields.length];
+        for (int i = 0; i < btreeTypeTraits.length; i++) {
+            btreeTypeTraits[i] = typeTraits[buddyBTreeFields[i]];
+        }
+
+        LSMTypeAwareTupleWriterFactory rtreeTupleWriterFactory = new LSMTypeAwareTupleWriterFactory(typeTraits, false);
+        LSMTypeAwareTupleWriterFactory btreeTupleWriterFactory = new LSMTypeAwareTupleWriterFactory(btreeTypeTraits,
+                true);
+
+        ITreeIndexFrameFactory rtreeInteriorFrameFactory = new RTreeNSMInteriorFrameFactory(rtreeTupleWriterFactory,
+                valueProviderFactories, rtreePolicyType);
+        ITreeIndexFrameFactory rtreeLeafFrameFactory = new RTreeNSMLeafFrameFactory(rtreeTupleWriterFactory,
+                valueProviderFactories, rtreePolicyType);
+
+        ITreeIndexFrameFactory btreeInteriorFrameFactory = new BTreeNSMInteriorFrameFactory(btreeTupleWriterFactory);
+        ITreeIndexFrameFactory btreeLeafFrameFactory = new BTreeNSMLeafFrameFactory(btreeTupleWriterFactory);
+
+        ITreeIndexMetaDataFrameFactory metaFrameFactory = new LIFOMetaDataFrameFactory();
+        LinkedListFreePageManagerFactory freePageManagerFactory = new LinkedListFreePageManagerFactory(diskBufferCache,
+                metaFrameFactory);
+
+        TreeIndexFactory<RTree> diskRTreeFactory = new RTreeFactory(diskBufferCache, diskFileMapProvider,
+                freePageManagerFactory, rtreeInteriorFrameFactory, rtreeLeafFrameFactory, rtreeCmpFactories,
+                typeTraits.length);
+        TreeIndexFactory<BTree> diskBTreeFactory = new BTreeFactory(diskBufferCache, diskFileMapProvider,
+                freePageManagerFactory, btreeInteriorFrameFactory, btreeLeafFrameFactory, btreeCmpFactories,
+                btreeTypeTraits.length);
+        int[] comparatorFields = { 0 };
+        IBinaryComparatorFactory[] linearizerArray = { linearizeCmpFactory };
+
+        int[] bloomFilterKeyFields = new int[btreeCmpFactories.length];
+        for (int i = 0; i < btreeCmpFactories.length; i++) {
+            bloomFilterKeyFields[i] = i;
+        }
+        BloomFilterFactory bloomFilterFactory = new BloomFilterFactory(diskBufferCache, diskFileMapProvider,
+                bloomFilterKeyFields);
+
+        ILSMIndexFileManager fileNameManager = new LSMRTreeFileManager(diskFileMapProvider, file, diskRTreeFactory,
+                diskBTreeFactory);
+        ExternalRTree lsmTree = new ExternalRTree(rtreeInteriorFrameFactory, rtreeLeafFrameFactory,
+                btreeInteriorFrameFactory, btreeLeafFrameFactory, fileNameManager, diskRTreeFactory, diskBTreeFactory,
+                bloomFilterFactory, bloomFilterFalsePositiveRate, diskFileMapProvider, typeTraits.length,
+                rtreeCmpFactories, btreeCmpFactories, linearizeCmpFactory, comparatorFields, linearizerArray,
+                mergePolicy, opTracker, ioScheduler, ioOpCallback, buddyBTreeFields, startWithVersion);
+        return lsmTree;
+    }
+
     public static ILinearizeComparatorFactory proposeBestLinearizer(ITypeTraits[] typeTraits, int numKeyFields)
             throws TreeIndexException {
         for (int i = 0; i < numKeyFields; i++) {
diff --git a/hyracks/hyracks-storage-common/src/main/java/edu/uci/ics/hyracks/storage/common/file/LocalResource.java b/hyracks/hyracks-storage-common/src/main/java/edu/uci/ics/hyracks/storage/common/file/LocalResource.java
index d91e02e..dd69e67 100644
--- a/hyracks/hyracks-storage-common/src/main/java/edu/uci/ics/hyracks/storage/common/file/LocalResource.java
+++ b/hyracks/hyracks-storage-common/src/main/java/edu/uci/ics/hyracks/storage/common/file/LocalResource.java
@@ -28,6 +28,9 @@
     public static final int LSMBTreeResource = 1;
     public static final int LSMRTreeResource = 2;
     public static final int LSMInvertedIndexResource = 3;
+    public static final int ExternalBTreeResource = 4;
+    public static final int ExternalRTreeResource = 5;
+    public static final int ExternalBTreeWithBuddyResource = 6;
 
     public LocalResource(long resourceId, String resourceName, int partition, int resourceType, Object object) {
         this.resourceId = resourceId;