Merge branch 'master' into raman/master_feeds
diff --git a/.gitignore b/.gitignore
index babae45..503b769 100644
--- a/.gitignore
+++ b/.gitignore
@@ -10,6 +10,8 @@
 expected
 teststore1
 teststore2
+dev1
+dev2
 derby.log
 hadoop-conf-tmp
 metastore_db
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 af65ebd..c9ef2f3 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
@@ -57,8 +57,8 @@
     INSERT_DELETE,
     INDEX_INSERT_DELETE,
     UPDATE,
-    INVERTED_INDEX_SEARCH,
-    FUZZY_INVERTED_INDEX_SEARCH,
+    SINGLE_PARTITION_INVERTED_INDEX_SEARCH,
+    LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH,
     PARTITIONINGSPLIT,
     EXTENSION_OPERATOR
 }
diff --git a/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/base/IPushRuntimeFactory.java b/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/base/IPushRuntimeFactory.java
index 12a5282..703bc1f 100644
--- a/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/base/IPushRuntimeFactory.java
+++ b/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/base/IPushRuntimeFactory.java
@@ -18,7 +18,8 @@
 
 import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
 import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 
 public interface IPushRuntimeFactory extends Serializable {
-    public IPushRuntime createPushRuntime(IHyracksTaskContext ctx) throws AlgebricksException;
+    public IPushRuntime createPushRuntime(IHyracksTaskContext ctx) throws AlgebricksException, HyracksDataException;
 }
diff --git a/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/operators/aggreg/NestedPlansAccumulatingAggregatorFactory.java b/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/operators/aggreg/NestedPlansAccumulatingAggregatorFactory.java
index c17b06f..8f2eaac 100644
--- a/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/operators/aggreg/NestedPlansAccumulatingAggregatorFactory.java
+++ b/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/operators/aggreg/NestedPlansAccumulatingAggregatorFactory.java
@@ -141,7 +141,7 @@
     }
 
     private IFrameWriter assemblePipeline(AlgebricksPipeline subplan, IFrameWriter writer, IHyracksTaskContext ctx)
-            throws AlgebricksException {
+            throws AlgebricksException, HyracksDataException {
         // plug the operators
         IFrameWriter start = writer;
         IPushRuntimeFactory[] runtimeFactories = subplan.getRuntimeFactories();
diff --git a/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/operators/base/AbstractOneInputOneOutputOneFramePushRuntime.java b/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/operators/base/AbstractOneInputOneOutputOneFramePushRuntime.java
index 7f5fead..082e98a 100644
--- a/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/operators/base/AbstractOneInputOneOutputOneFramePushRuntime.java
+++ b/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/operators/base/AbstractOneInputOneOutputOneFramePushRuntime.java
@@ -85,7 +85,7 @@
         }
     }
 
-    protected final void initAccessAppend(IHyracksTaskContext ctx) {
+    protected final void initAccessAppend(IHyracksTaskContext ctx) throws HyracksDataException {
         // if (allocFrame) {
         frame = ctx.allocateFrame();
         appender = new FrameTupleAppender(ctx.getFrameSize());
@@ -94,7 +94,7 @@
         tAccess = new FrameTupleAccessor(ctx.getFrameSize(), inputRecordDesc);
     }
 
-    protected final void initAccessAppendRef(IHyracksTaskContext ctx) {
+    protected final void initAccessAppendRef(IHyracksTaskContext ctx) throws HyracksDataException {
         initAccessAppend(ctx);
         tRef = new FrameTupleReference();
     }
diff --git a/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/operators/base/AbstractOneInputOneOutputRuntimeFactory.java b/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/operators/base/AbstractOneInputOneOutputRuntimeFactory.java
index 084b085..43270b6 100644
--- a/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/operators/base/AbstractOneInputOneOutputRuntimeFactory.java
+++ b/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/operators/base/AbstractOneInputOneOutputRuntimeFactory.java
@@ -18,6 +18,7 @@
 import edu.uci.ics.hyracks.algebricks.runtime.base.IPushRuntime;
 import edu.uci.ics.hyracks.algebricks.runtime.base.IPushRuntimeFactory;
 import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 
 public abstract class AbstractOneInputOneOutputRuntimeFactory implements IPushRuntimeFactory {
 
@@ -30,11 +31,11 @@
     }
 
     @Override
-    public IPushRuntime createPushRuntime(IHyracksTaskContext ctx) throws AlgebricksException {
+    public IPushRuntime createPushRuntime(IHyracksTaskContext ctx) throws AlgebricksException, HyracksDataException {
         return createOneOutputPushRuntime(ctx);
     }
 
     public abstract AbstractOneInputOneOutputPushRuntime createOneOutputPushRuntime(IHyracksTaskContext ctx)
-            throws AlgebricksException;
+            throws AlgebricksException, HyracksDataException;
 
 }
diff --git a/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/operators/meta/PipelineAssembler.java b/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/operators/meta/PipelineAssembler.java
index cef05d8..5bcf933 100644
--- a/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/operators/meta/PipelineAssembler.java
+++ b/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/operators/meta/PipelineAssembler.java
@@ -20,6 +20,7 @@
 import edu.uci.ics.hyracks.api.comm.IFrameWriter;
 import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 
 public class PipelineAssembler {
 
@@ -40,7 +41,8 @@
         this.outputArity = outputArity;
     }
 
-    public IFrameWriter assemblePipeline(IFrameWriter writer, IHyracksTaskContext ctx) throws AlgebricksException {
+    public IFrameWriter assemblePipeline(IFrameWriter writer, IHyracksTaskContext ctx) throws AlgebricksException,
+            HyracksDataException {
         // plug the operators
         IFrameWriter start = writer;// this.writer;
         for (int i = pipeline.getRuntimeFactories().length - 1; i >= 0; i--) {
diff --git a/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/operators/meta/SubplanRuntimeFactory.java b/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/operators/meta/SubplanRuntimeFactory.java
index 6e43bd1..0499684 100644
--- a/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/operators/meta/SubplanRuntimeFactory.java
+++ b/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/operators/meta/SubplanRuntimeFactory.java
@@ -67,7 +67,7 @@
 
     @Override
     public AbstractOneInputOneOutputPushRuntime createOneOutputPushRuntime(final IHyracksTaskContext ctx)
-            throws AlgebricksException {
+            throws AlgebricksException, HyracksDataException {
 
         RecordDescriptor pipelineOutputRecordDescriptor = null;
 
diff --git a/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/operators/std/EmptyTupleSourceRuntimeFactory.java b/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/operators/std/EmptyTupleSourceRuntimeFactory.java
index 38eddd1..745fdf6 100644
--- a/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/operators/std/EmptyTupleSourceRuntimeFactory.java
+++ b/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/operators/std/EmptyTupleSourceRuntimeFactory.java
@@ -38,7 +38,7 @@
     }
 
     @Override
-    public IPushRuntime createPushRuntime(final IHyracksTaskContext ctx) {
+    public IPushRuntime createPushRuntime(final IHyracksTaskContext ctx) throws HyracksDataException {
         return new AbstractOneInputSourcePushRuntime() {
 
             private ByteBuffer frame = ctx.allocateFrame();
diff --git a/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/operators/std/NestedTupleSourceRuntimeFactory.java b/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/operators/std/NestedTupleSourceRuntimeFactory.java
index dc155a2..5486fae 100644
--- a/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/operators/std/NestedTupleSourceRuntimeFactory.java
+++ b/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/operators/std/NestedTupleSourceRuntimeFactory.java
@@ -35,13 +35,13 @@
     }
 
     @Override
-    public IPushRuntime createPushRuntime(IHyracksTaskContext ctx) {
+    public IPushRuntime createPushRuntime(IHyracksTaskContext ctx) throws HyracksDataException {
         return new NestedTupleSourceRuntime(ctx);
     }
 
     public static class NestedTupleSourceRuntime extends AbstractOneInputOneOutputOneFramePushRuntime {
 
-        public NestedTupleSourceRuntime(IHyracksTaskContext ctx) {
+        public NestedTupleSourceRuntime(IHyracksTaskContext ctx) throws HyracksDataException {
             initAccessAppend(ctx);
         }
 
diff --git a/hivesterix/hivesterix-dist/pom.xml b/hivesterix/hivesterix-dist/pom.xml
index 2adcc7c..8ecdfe1 100644
--- a/hivesterix/hivesterix-dist/pom.xml
+++ b/hivesterix/hivesterix-dist/pom.xml
@@ -14,9 +14,7 @@
  !-->
 <project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
 	<modelVersion>4.0.0</modelVersion>
-	<groupId>edu.uci.ics.hyracks</groupId>
 	<artifactId>hivesterix-dist</artifactId>
-	<version>0.2.7-SNAPSHOT</version>
 	<name>hivesterix-dist</name>
 
 	<parent>
diff --git a/hivesterix/hivesterix-dist/src/main/java/edu/uci/ics/hivesterix/runtime/exec/HyracksExecutionEngine.java b/hivesterix/hivesterix-dist/src/main/java/edu/uci/ics/hivesterix/runtime/exec/HyracksExecutionEngine.java
index 0c2416d..d3bcaca 100644
--- a/hivesterix/hivesterix-dist/src/main/java/edu/uci/ics/hivesterix/runtime/exec/HyracksExecutionEngine.java
+++ b/hivesterix/hivesterix-dist/src/main/java/edu/uci/ics/hivesterix/runtime/exec/HyracksExecutionEngine.java
@@ -12,598 +12,601 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package edu.uci.ics.hivesterix.runtime.exec;

-

-import java.io.BufferedReader;

-import java.io.FileInputStream;

-import java.io.InputStream;

-import java.io.InputStreamReader;

-import java.io.PrintWriter;

-import java.io.Serializable;

-import java.net.InetAddress;

-import java.util.ArrayList;

-import java.util.HashMap;

-import java.util.Iterator;

-import java.util.List;

-import java.util.Map;

-import java.util.Map.Entry;

-import java.util.Properties;

-import java.util.Set;

-

-import org.apache.commons.logging.Log;

-import org.apache.commons.logging.LogFactory;

-import org.apache.hadoop.hive.conf.HiveConf;

-import org.apache.hadoop.hive.ql.exec.ConditionalTask;

-import org.apache.hadoop.hive.ql.exec.FileSinkOperator;

-import org.apache.hadoop.hive.ql.exec.MapRedTask;

-import org.apache.hadoop.hive.ql.exec.Operator;

-import org.apache.hadoop.hive.ql.exec.TableScanOperator;

-import org.apache.hadoop.hive.ql.exec.Task;

-import org.apache.hadoop.hive.ql.plan.DynamicPartitionCtx;

-import org.apache.hadoop.hive.ql.plan.FetchWork;

-import org.apache.hadoop.hive.ql.plan.FileSinkDesc;

-import org.apache.hadoop.hive.ql.plan.MapredLocalWork;

-import org.apache.hadoop.hive.ql.plan.MapredWork;

-import org.apache.hadoop.hive.ql.plan.PartitionDesc;

-import org.apache.hadoop.hive.ql.plan.TableScanDesc;

-

-import edu.uci.ics.hivesterix.common.config.ConfUtil;

-import edu.uci.ics.hivesterix.logical.expression.HiveExpressionTypeComputer;

-import edu.uci.ics.hivesterix.logical.expression.HiveMergeAggregationExpressionFactory;

-import edu.uci.ics.hivesterix.logical.expression.HiveNullableTypeComputer;

-import edu.uci.ics.hivesterix.logical.expression.HivePartialAggregationTypeComputer;

-import edu.uci.ics.hivesterix.logical.plan.HiveAlgebricksTranslator;

-import edu.uci.ics.hivesterix.logical.plan.HiveLogicalPlanAndMetaData;

-import edu.uci.ics.hivesterix.optimizer.rulecollections.HiveRuleCollections;

-import edu.uci.ics.hivesterix.runtime.factory.evaluator.HiveExpressionRuntimeProvider;

-import edu.uci.ics.hivesterix.runtime.factory.nullwriter.HiveNullWriterFactory;

-import edu.uci.ics.hivesterix.runtime.inspector.HiveBinaryBooleanInspectorFactory;

-import edu.uci.ics.hivesterix.runtime.inspector.HiveBinaryIntegerInspectorFactory;

-import edu.uci.ics.hivesterix.runtime.jobgen.HiveConnectorPolicyAssignmentPolicy;

-import edu.uci.ics.hivesterix.runtime.jobgen.HiveConnectorPolicyAssignmentPolicy.Policy;

-import edu.uci.ics.hivesterix.runtime.provider.HiveBinaryComparatorFactoryProvider;

-import edu.uci.ics.hivesterix.runtime.provider.HiveBinaryHashFunctionFactoryProvider;

-import edu.uci.ics.hivesterix.runtime.provider.HiveBinaryHashFunctionFamilyProvider;

-import edu.uci.ics.hivesterix.runtime.provider.HiveNormalizedKeyComputerFactoryProvider;

-import edu.uci.ics.hivesterix.runtime.provider.HivePrinterFactoryProvider;

-import edu.uci.ics.hivesterix.runtime.provider.HiveSerializerDeserializerProvider;

-import edu.uci.ics.hivesterix.runtime.provider.HiveTypeTraitProvider;

-import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksAbsolutePartitionConstraint;

-import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;

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

-import edu.uci.ics.hyracks.algebricks.compiler.api.HeuristicCompilerFactoryBuilder;

-import edu.uci.ics.hyracks.algebricks.compiler.api.HeuristicCompilerFactoryBuilder.DefaultOptimizationContextFactory;

-import edu.uci.ics.hyracks.algebricks.compiler.api.ICompiler;

-import edu.uci.ics.hyracks.algebricks.compiler.api.ICompilerFactory;

-import edu.uci.ics.hyracks.algebricks.compiler.rewriter.rulecontrollers.SequentialFixpointRuleController;

-import edu.uci.ics.hyracks.algebricks.compiler.rewriter.rulecontrollers.SequentialOnceRuleController;

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

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

-import edu.uci.ics.hyracks.algebricks.core.algebra.prettyprint.LogicalOperatorPrettyPrintVisitor;

-import edu.uci.ics.hyracks.algebricks.core.algebra.prettyprint.PlanPrettyPrinter;

-import edu.uci.ics.hyracks.algebricks.core.rewriter.base.AbstractRuleController;

-import edu.uci.ics.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;

-import edu.uci.ics.hyracks.algebricks.core.rewriter.base.PhysicalOptimizationConfig;

-import edu.uci.ics.hyracks.api.client.HyracksConnection;

-import edu.uci.ics.hyracks.api.client.IHyracksClientConnection;

-import edu.uci.ics.hyracks.api.job.JobId;

-import edu.uci.ics.hyracks.api.job.JobSpecification;

-

-@SuppressWarnings({ "rawtypes", "unchecked" })

-public class HyracksExecutionEngine implements IExecutionEngine {

-

-    private static final Log LOG = LogFactory.getLog(HyracksExecutionEngine.class.getName());

-    private static final String clusterPropertiesPath = "conf/cluster.properties";

-    private static final String masterFilePath = "conf/master";

-

-    private static List<Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>> DEFAULT_LOGICAL_REWRITES = new ArrayList<Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>>();

-    private static List<Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>> DEFAULT_PHYSICAL_REWRITES = new ArrayList<Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>>();

-    static {

-        SequentialFixpointRuleController seqCtrlNoDfs = new SequentialFixpointRuleController(false);

-        SequentialFixpointRuleController seqCtrlFullDfs = new SequentialFixpointRuleController(true);

-        SequentialOnceRuleController seqOnceCtrl = new SequentialOnceRuleController(true);

-        DEFAULT_LOGICAL_REWRITES.add(new Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>(seqCtrlFullDfs,

-                HiveRuleCollections.NORMALIZATION));

-        DEFAULT_LOGICAL_REWRITES.add(new Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>(seqCtrlNoDfs,

-                HiveRuleCollections.COND_PUSHDOWN_AND_JOIN_INFERENCE));

-        DEFAULT_LOGICAL_REWRITES.add(new Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>(seqCtrlFullDfs,

-                HiveRuleCollections.LOAD_FIELDS));

-        DEFAULT_LOGICAL_REWRITES.add(new Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>(seqCtrlNoDfs,

-                HiveRuleCollections.OP_PUSHDOWN));

-        DEFAULT_LOGICAL_REWRITES.add(new Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>(seqOnceCtrl,

-                HiveRuleCollections.DATA_EXCHANGE));

-        DEFAULT_LOGICAL_REWRITES.add(new Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>(seqCtrlNoDfs,

-                HiveRuleCollections.CONSOLIDATION));

-

-        DEFAULT_PHYSICAL_REWRITES.add(new Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>(seqOnceCtrl,

-                HiveRuleCollections.PHYSICAL_PLAN_REWRITES));

-        DEFAULT_PHYSICAL_REWRITES.add(new Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>(seqOnceCtrl,

-                HiveRuleCollections.prepareJobGenRules));

-    }

-

-    /**

-     * static configurations for compiler

-     */

-    private HeuristicCompilerFactoryBuilder builder;

-

-    /**

-     * compiler

-     */

-    private ICompiler compiler;

-

-    /**

-     * physical optimization config

-     */

-    private PhysicalOptimizationConfig physicalOptimizationConfig;

-

-    /**

-     * final ending operators

-     */

-    private List<Operator> leaveOps = new ArrayList<Operator>();

-

-    /**

-     * tasks that are already visited

-     */

-    private Map<Task<? extends Serializable>, Boolean> tasksVisited = new HashMap<Task<? extends Serializable>, Boolean>();

-

-    /**

-     * hyracks job spec

-     */

-    private JobSpecification jobSpec;

-

-    /**

-     * hive configuration

-     */

-    private HiveConf conf;

-

-    /**

-     * plan printer

-     */

-    private PrintWriter planPrinter;

-

-    /**

-     * properties

-     */

-    private Properties clusterProps;

-

-    /**

-     * the Hyracks client connection

-     */

-    private IHyracksClientConnection hcc;

-

-    public HyracksExecutionEngine(HiveConf conf) {

-        this.conf = conf;

-        init(conf);

-    }

-

-    public HyracksExecutionEngine(HiveConf conf, PrintWriter planPrinter) {

-        this.conf = conf;

-        this.planPrinter = planPrinter;

-        init(conf);

-    }

-

-    private void init(HiveConf conf) {

-        builder = new HeuristicCompilerFactoryBuilder(DefaultOptimizationContextFactory.INSTANCE);

-        builder.setLogicalRewrites(DEFAULT_LOGICAL_REWRITES);

-        builder.setPhysicalRewrites(DEFAULT_PHYSICAL_REWRITES);

-        builder.setIMergeAggregationExpressionFactory(HiveMergeAggregationExpressionFactory.INSTANCE);

-        builder.setExpressionTypeComputer(HiveExpressionTypeComputer.INSTANCE);

-        builder.setNullableTypeComputer(HiveNullableTypeComputer.INSTANCE);

-

-        long memSizeExternalGby = conf.getLong("hive.algebricks.groupby.external.memory", 268435456);

-        long memSizeExternalSort = conf.getLong("hive.algebricks.sort.memory", 536870912);

-        int frameSize = conf.getInt("hive.algebricks.framesize", 32768);

-

-        physicalOptimizationConfig = new PhysicalOptimizationConfig();

-        int frameLimitExtGby = (int) (memSizeExternalGby / frameSize);

-        physicalOptimizationConfig.setMaxFramesExternalGroupBy(frameLimitExtGby);

-        int frameLimitExtSort = (int) (memSizeExternalSort / frameSize);

-        physicalOptimizationConfig.setMaxFramesExternalSort(frameLimitExtSort);

-        builder.setPhysicalOptimizationConfig(physicalOptimizationConfig);

-    }

-

-    @Override

-    public int compileJob(List<Task<? extends Serializable>> rootTasks) {

-        // clean up

-        leaveOps.clear();

-        tasksVisited.clear();

-        jobSpec = null;

-

-        HashMap<String, PartitionDesc> aliasToPath = new HashMap<String, PartitionDesc>();

-        List<Operator> rootOps = generateRootOperatorDAG(rootTasks, aliasToPath);

-

-        // get all leave Ops

-        getLeaves(rootOps, leaveOps);

-

-        HiveAlgebricksTranslator translator = new HiveAlgebricksTranslator();

-        try {

-            translator.translate(rootOps, null, aliasToPath);

-

-            ILogicalPlan plan = translator.genLogicalPlan();

-

-            if (plan.getRoots() != null && plan.getRoots().size() > 0 && plan.getRoots().get(0).getValue() != null) {

-                translator.printOperators();

-                ILogicalPlanAndMetadata planAndMetadata = new HiveLogicalPlanAndMetaData(plan,

-                        translator.getMetadataProvider());

-

-                ICompilerFactory compilerFactory = builder.create();

-                compiler = compilerFactory.createCompiler(planAndMetadata.getPlan(),

-                        planAndMetadata.getMetadataProvider(), translator.getVariableCounter());

-

-                // run optimization and re-writing rules for Hive plan

-                compiler.optimize();

-

-                // print optimized plan

-                LogicalOperatorPrettyPrintVisitor pvisitor = new LogicalOperatorPrettyPrintVisitor();

-                StringBuilder buffer = new StringBuilder();

-                PlanPrettyPrinter.printPlan(plan, buffer, pvisitor, 0);

-                String planStr = buffer.toString();

-                System.out.println(planStr);

-

-                if (planPrinter != null)

-                    planPrinter.print(planStr);

-            }

-        } catch (Exception e) {

-            e.printStackTrace();

-            return 1;

-        }

-

-        return 0;

-    }

-

-    private void codeGen() throws AlgebricksException {

-        try {

-            // number of cpu cores in the cluster

-            builder.setClusterLocations(new AlgebricksAbsolutePartitionConstraint(ConfUtil.getNCs()));

-        } catch (Exception e) {

-            throw new AlgebricksException(e);

-        }

-        // builder.setClusterTopology(ConfUtil.getClusterTopology());

-        builder.setBinaryBooleanInspectorFactory(HiveBinaryBooleanInspectorFactory.INSTANCE);

-        builder.setBinaryIntegerInspectorFactory(HiveBinaryIntegerInspectorFactory.INSTANCE);

-        builder.setComparatorFactoryProvider(HiveBinaryComparatorFactoryProvider.INSTANCE);

-        builder.setExpressionRuntimeProvider(HiveExpressionRuntimeProvider.INSTANCE);

-        builder.setHashFunctionFactoryProvider(HiveBinaryHashFunctionFactoryProvider.INSTANCE);

-        builder.setPrinterProvider(HivePrinterFactoryProvider.INSTANCE);

-        builder.setSerializerDeserializerProvider(HiveSerializerDeserializerProvider.INSTANCE);

-        builder.setNullWriterFactory(HiveNullWriterFactory.INSTANCE);

-        builder.setNormalizedKeyComputerFactoryProvider(HiveNormalizedKeyComputerFactoryProvider.INSTANCE);

-        builder.setPartialAggregationTypeComputer(HivePartialAggregationTypeComputer.INSTANCE);

-        builder.setTypeTraitProvider(HiveTypeTraitProvider.INSTANCE);

-        builder.setHashFunctionFamilyProvider(HiveBinaryHashFunctionFamilyProvider.INSTANCE);

-

-        jobSpec = compiler.createJob(null, null);

-

-        // set the policy

-        String policyStr = conf.get("hive.hyracks.connectorpolicy");

-        if (policyStr == null)

-            policyStr = "PIPELINING";

-        Policy policyValue = Policy.valueOf(policyStr);

-        jobSpec.setConnectorPolicyAssignmentPolicy(new HiveConnectorPolicyAssignmentPolicy(policyValue));

-        jobSpec.setUseConnectorPolicyForScheduling(false);

-    }

-

-    @Override

-    public int executeJob() {

-        try {

-            codeGen();

-            executeHyracksJob(jobSpec);

-        } catch (Exception e) {

-            e.printStackTrace();

-            return 1;

-        }

-        return 0;

-    }

-

-    private List<Operator> generateRootOperatorDAG(List<Task<? extends Serializable>> rootTasks,

-            HashMap<String, PartitionDesc> aliasToPath) {

-

-        List<Operator> rootOps = new ArrayList<Operator>();

-        List<Task<? extends Serializable>> toDelete = new ArrayList<Task<? extends Serializable>>();

-        tasksVisited.clear();

-

-        for (int i = rootTasks.size() - 1; i >= 0; i--) {

-            /**

-             * list of map-reduce tasks

-             */

-            Task<? extends Serializable> task = rootTasks.get(i);

-

-            if (task instanceof MapRedTask) {

-                List<Operator> mapRootOps = articulateMapReduceOperators(task, rootOps, aliasToPath, rootTasks);

-                if (i == 0)

-                    rootOps.addAll(mapRootOps);

-                else {

-                    List<Operator> leaves = new ArrayList<Operator>();

-                    getLeaves(rootOps, leaves);

-

-                    List<Operator> mapChildren = new ArrayList<Operator>();

-                    for (Operator childMap : mapRootOps) {

-                        if (childMap instanceof TableScanOperator) {

-                            TableScanDesc topDesc = (TableScanDesc) childMap.getConf();

-                            if (topDesc == null)

-                                mapChildren.add(childMap);

-                            else {

-                                rootOps.add(childMap);

-                            }

-                        } else

-                            mapChildren.add(childMap);

-                    }

-

-                    if (mapChildren.size() > 0) {

-                        for (Operator leaf : leaves)

-                            leaf.setChildOperators(mapChildren);

-                        for (Operator child : mapChildren)

-                            child.setParentOperators(leaves);

-                    }

-                }

-

-                MapredWork mr = (MapredWork) task.getWork();

-                HashMap<String, PartitionDesc> map = mr.getAliasToPartnInfo();

-

-                addAliasToPartition(aliasToPath, map);

-                toDelete.add(task);

-            }

-        }

-

-        for (Task<? extends Serializable> task : toDelete)

-            rootTasks.remove(task);

-

-        return rootOps;

-    }

-

-    private void addAliasToPartition(HashMap<String, PartitionDesc> aliasToPath, HashMap<String, PartitionDesc> map) {

-        Iterator<String> keys = map.keySet().iterator();

-        while (keys.hasNext()) {

-            String key = keys.next();

-            PartitionDesc part = map.get(key);

-            String[] names = key.split(":");

-            for (String name : names) {

-                aliasToPath.put(name, part);

-            }

-        }

-    }

-

-    private List<Operator> articulateMapReduceOperators(Task task, List<Operator> rootOps,

-            HashMap<String, PartitionDesc> aliasToPath, List<Task<? extends Serializable>> rootTasks) {

-        // System.out.println("!"+task.getName());

-        if (!(task instanceof MapRedTask)) {

-            if (!(task instanceof ConditionalTask)) {

-                rootTasks.add(task);

-                return null;

-            } else {

-                // remove map-reduce branches in condition task

-                ConditionalTask condition = (ConditionalTask) task;

-                List<Task<? extends Serializable>> branches = condition.getListTasks();

-                for (int i = branches.size() - 1; i >= 0; i--) {

-                    Task branch = branches.get(i);

-                    if (branch instanceof MapRedTask) {

-                        return articulateMapReduceOperators(branch, rootOps, aliasToPath, rootTasks);

-                    }

-                }

-                rootTasks.add(task);

-                return null;

-            }

-        }

-

-        MapredWork mr = (MapredWork) task.getWork();

-        HashMap<String, PartitionDesc> map = mr.getAliasToPartnInfo();

-

-        // put all aliasToParitionDesc mapping into the map

-        addAliasToPartition(aliasToPath, map);

-

-        MapRedTask mrtask = (MapRedTask) task;

-        MapredWork work = (MapredWork) mrtask.getWork();

-        HashMap<String, Operator<? extends Serializable>> operators = work.getAliasToWork();

-

-        Set entries = operators.entrySet();

-        Iterator<Entry<String, Operator>> iterator = entries.iterator();

-        List<Operator> mapRootOps = new ArrayList<Operator>();

-

-        // get map root operators

-        while (iterator.hasNext()) {

-            Operator next = iterator.next().getValue();

-            if (!mapRootOps.contains(next)) {

-                // clear that only for the case of union

-                mapRootOps.add(next);

-            }

-        }

-

-        // get map local work

-        MapredLocalWork localWork = work.getMapLocalWork();

-        if (localWork != null) {

-            HashMap<String, Operator<? extends Serializable>> localOperators = localWork.getAliasToWork();

-

-            Set localEntries = localOperators.entrySet();

-            Iterator<Entry<String, Operator>> localIterator = localEntries.iterator();

-            while (localIterator.hasNext()) {

-                mapRootOps.add(localIterator.next().getValue());

-            }

-

-            HashMap<String, FetchWork> localFetch = localWork.getAliasToFetchWork();

-            Set localFetchEntries = localFetch.entrySet();

-            Iterator<Entry<String, FetchWork>> localFetchIterator = localFetchEntries.iterator();

-            while (localFetchIterator.hasNext()) {

-                Entry<String, FetchWork> fetchMap = localFetchIterator.next();

-                FetchWork fetch = fetchMap.getValue();

-                String alias = fetchMap.getKey();

-                List<PartitionDesc> dirPart = fetch.getPartDesc();

-

-                // temporary hack: put the first partitionDesc into the map

-                aliasToPath.put(alias, dirPart.get(0));

-            }

-        }

-

-        Boolean visited = tasksVisited.get(task);

-        if (visited != null && visited.booleanValue() == true) {

-            return mapRootOps;

-        }

-

-        // do that only for union operator

-        for (Operator op : mapRootOps)

-            if (op.getParentOperators() != null)

-                op.getParentOperators().clear();

-

-        List<Operator> mapLeaves = new ArrayList<Operator>();

-        downToLeaves(mapRootOps, mapLeaves);

-        List<Operator> reduceOps = new ArrayList<Operator>();

-

-        if (work.getReducer() != null)

-            reduceOps.add(work.getReducer());

-

-        for (Operator mapLeaf : mapLeaves) {

-            mapLeaf.setChildOperators(reduceOps);

-        }

-

-        for (Operator reduceOp : reduceOps) {

-            if (reduceOp != null)

-                reduceOp.setParentOperators(mapLeaves);

-        }

-

-        List<Operator> leafs = new ArrayList<Operator>();

-        if (reduceOps.size() > 0) {

-            downToLeaves(reduceOps, leafs);

-        } else {

-            leafs = mapLeaves;

-        }

-

-        List<Operator> mapChildren = new ArrayList<Operator>();

-        if (task.getChildTasks() != null && task.getChildTasks().size() > 0) {

-            for (Object child : task.getChildTasks()) {

-                List<Operator> childMapOps = articulateMapReduceOperators((Task) child, rootOps, aliasToPath, rootTasks);

-                if (childMapOps == null)

-                    continue;

-

-                for (Operator childMap : childMapOps) {

-                    if (childMap instanceof TableScanOperator) {

-                        TableScanDesc topDesc = (TableScanDesc) childMap.getConf();

-                        if (topDesc == null)

-                            mapChildren.add(childMap);

-                        else {

-                            rootOps.add(childMap);

-                        }

-                    } else {

-                        // if not table scan, add the child

-                        mapChildren.add(childMap);

-                    }

-                }

-            }

-

-            if (mapChildren.size() > 0) {

-                int i = 0;

-                for (Operator leaf : leafs) {

-                    if (leaf.getChildOperators() == null || leaf.getChildOperators().size() == 0)

-                        leaf.setChildOperators(new ArrayList<Operator>());

-                    leaf.getChildOperators().add(mapChildren.get(i));

-                    i++;

-                }

-                i = 0;

-                for (Operator child : mapChildren) {

-                    if (child.getParentOperators() == null || child.getParentOperators().size() == 0)

-                        child.setParentOperators(new ArrayList<Operator>());

-                    child.getParentOperators().add(leafs.get(i));

-                    i++;

-                }

-            }

-        }

-

-        // mark this task as visited

-        this.tasksVisited.put(task, true);

-        return mapRootOps;

-    }

-

-    /**

-     * down to leaf nodes

-     * 

-     * @param ops

-     * @param leaves

-     */

-    private void downToLeaves(List<Operator> ops, List<Operator> leaves) {

-

-        // Operator currentOp;

-        for (Operator op : ops) {

-            if (op != null && op.getChildOperators() != null && op.getChildOperators().size() > 0) {

-                downToLeaves(op.getChildOperators(), leaves);

-            } else {

-                if (op != null && leaves.indexOf(op) < 0)

-                    leaves.add(op);

-            }

-        }

-    }

-

-    private void getLeaves(List<Operator> roots, List<Operator> currentLeaves) {

-        for (Operator op : roots) {

-            List<Operator> children = op.getChildOperators();

-            if (children == null || children.size() <= 0) {

-                currentLeaves.add(op);

-            } else {

-                getLeaves(children, currentLeaves);

-            }

-        }

-    }

-

-    private void executeHyracksJob(JobSpecification job) throws Exception {

-

-        /**

-         * load the properties file if it is not loaded

-         */

-        if (clusterProps == null) {

-            clusterProps = new Properties();

-            InputStream confIn = new FileInputStream(clusterPropertiesPath);

-            clusterProps.load(confIn);

-            confIn.close();

-        }

-

-        if (hcc == null) {

-            BufferedReader ipReader = new BufferedReader(new InputStreamReader(new FileInputStream(masterFilePath)));

-            String masterNode = ipReader.readLine();

-            ipReader.close();

-

-            InetAddress[] ips = InetAddress.getAllByName(masterNode);

-            int port = Integer.parseInt(clusterProps.getProperty("CC_CLIENTPORT"));

-            for (InetAddress ip : ips) {

-                if (ip.getAddress().length <= 4) {

-                    try {

-                        hcc = new HyracksConnection(ip.getHostAddress(), port);

-                        break;

-                    } catch (Exception e) {

-                        continue;

-                    }

-                }

-            }

-        }

-

-        long start = System.currentTimeMillis();

-        JobId jobId = hcc.startJob(job);

-        hcc.waitForCompletion(jobId);

-

-        // System.out.println("job finished: " + jobId.toString());

-        // call all leave nodes to end

-        for (Operator leaf : leaveOps) {

-            jobClose(leaf);

-        }

-

-        long end = System.currentTimeMillis();

-        System.err.println(start + " " + end + " " + (end - start));

-    }

-

-    /**

-     * mv to final directory on hdfs (not real final)

-     * 

-     * @param leaf

-     * @throws Exception

-     */

-    private void jobClose(Operator leaf) throws Exception {

-        FileSinkOperator fsOp = (FileSinkOperator) leaf;

-        FileSinkDesc desc = fsOp.getConf();

-        boolean isNativeTable = !desc.getTableInfo().isNonNative();

-        if ((conf != null) && isNativeTable) {

-            String specPath = desc.getDirName();

-            DynamicPartitionCtx dpCtx = desc.getDynPartCtx();

-            // for 0.7.0

-            fsOp.mvFileToFinalPath(specPath, conf, true, LOG, dpCtx);

-            // for 0.8.0

-            // Utilities.mvFileToFinalPath(specPath, conf, true, LOG, dpCtx,

-            // desc);

-        }

-    }

-}

+package edu.uci.ics.hivesterix.runtime.exec;
+
+import java.io.BufferedReader;
+import java.io.FileInputStream;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.io.PrintWriter;
+import java.io.Serializable;
+import java.net.InetAddress;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Properties;
+import java.util.Set;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.ql.exec.ConditionalTask;
+import org.apache.hadoop.hive.ql.exec.FileSinkOperator;
+import org.apache.hadoop.hive.ql.exec.MapRedTask;
+import org.apache.hadoop.hive.ql.exec.Operator;
+import org.apache.hadoop.hive.ql.exec.TableScanOperator;
+import org.apache.hadoop.hive.ql.exec.Task;
+import org.apache.hadoop.hive.ql.plan.DynamicPartitionCtx;
+import org.apache.hadoop.hive.ql.plan.FetchWork;
+import org.apache.hadoop.hive.ql.plan.FileSinkDesc;
+import org.apache.hadoop.hive.ql.plan.MapredLocalWork;
+import org.apache.hadoop.hive.ql.plan.MapredWork;
+import org.apache.hadoop.hive.ql.plan.PartitionDesc;
+import org.apache.hadoop.hive.ql.plan.TableScanDesc;
+
+import edu.uci.ics.hivesterix.common.config.ConfUtil;
+import edu.uci.ics.hivesterix.logical.expression.HiveExpressionTypeComputer;
+import edu.uci.ics.hivesterix.logical.expression.HiveMergeAggregationExpressionFactory;
+import edu.uci.ics.hivesterix.logical.expression.HiveNullableTypeComputer;
+import edu.uci.ics.hivesterix.logical.expression.HivePartialAggregationTypeComputer;
+import edu.uci.ics.hivesterix.logical.plan.HiveAlgebricksTranslator;
+import edu.uci.ics.hivesterix.logical.plan.HiveLogicalPlanAndMetaData;
+import edu.uci.ics.hivesterix.optimizer.rulecollections.HiveRuleCollections;
+import edu.uci.ics.hivesterix.runtime.factory.evaluator.HiveExpressionRuntimeProvider;
+import edu.uci.ics.hivesterix.runtime.factory.nullwriter.HiveNullWriterFactory;
+import edu.uci.ics.hivesterix.runtime.inspector.HiveBinaryBooleanInspectorFactory;
+import edu.uci.ics.hivesterix.runtime.inspector.HiveBinaryIntegerInspectorFactory;
+import edu.uci.ics.hivesterix.runtime.jobgen.HiveConnectorPolicyAssignmentPolicy;
+import edu.uci.ics.hivesterix.runtime.jobgen.HiveConnectorPolicyAssignmentPolicy.Policy;
+import edu.uci.ics.hivesterix.runtime.provider.HiveBinaryComparatorFactoryProvider;
+import edu.uci.ics.hivesterix.runtime.provider.HiveBinaryHashFunctionFactoryProvider;
+import edu.uci.ics.hivesterix.runtime.provider.HiveBinaryHashFunctionFamilyProvider;
+import edu.uci.ics.hivesterix.runtime.provider.HiveNormalizedKeyComputerFactoryProvider;
+import edu.uci.ics.hivesterix.runtime.provider.HivePrinterFactoryProvider;
+import edu.uci.ics.hivesterix.runtime.provider.HiveSerializerDeserializerProvider;
+import edu.uci.ics.hivesterix.runtime.provider.HiveTypeTraitProvider;
+import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksAbsolutePartitionConstraint;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.common.utils.Pair;
+import edu.uci.ics.hyracks.algebricks.compiler.api.HeuristicCompilerFactoryBuilder;
+import edu.uci.ics.hyracks.algebricks.compiler.api.HeuristicCompilerFactoryBuilder.DefaultOptimizationContextFactory;
+import edu.uci.ics.hyracks.algebricks.compiler.api.ICompiler;
+import edu.uci.ics.hyracks.algebricks.compiler.api.ICompilerFactory;
+import edu.uci.ics.hyracks.algebricks.compiler.rewriter.rulecontrollers.SequentialFixpointRuleController;
+import edu.uci.ics.hyracks.algebricks.compiler.rewriter.rulecontrollers.SequentialOnceRuleController;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalPlan;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalPlanAndMetadata;
+import edu.uci.ics.hyracks.algebricks.core.algebra.prettyprint.LogicalOperatorPrettyPrintVisitor;
+import edu.uci.ics.hyracks.algebricks.core.algebra.prettyprint.PlanPrettyPrinter;
+import edu.uci.ics.hyracks.algebricks.core.rewriter.base.AbstractRuleController;
+import edu.uci.ics.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+import edu.uci.ics.hyracks.algebricks.core.rewriter.base.PhysicalOptimizationConfig;
+import edu.uci.ics.hyracks.api.client.HyracksConnection;
+import edu.uci.ics.hyracks.api.client.IHyracksClientConnection;
+import edu.uci.ics.hyracks.api.job.JobId;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
+
+@SuppressWarnings({ "rawtypes", "unchecked" })
+public class HyracksExecutionEngine implements IExecutionEngine {
+
+    private static final Log LOG = LogFactory.getLog(HyracksExecutionEngine.class.getName());
+    private static final String clusterPropertiesPath = "conf/cluster.properties";
+    private static final String masterFilePath = "conf/master";
+
+    private static List<Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>> DEFAULT_LOGICAL_REWRITES = new ArrayList<Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>>();
+    private static List<Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>> DEFAULT_PHYSICAL_REWRITES = new ArrayList<Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>>();
+    static {
+        SequentialFixpointRuleController seqCtrlNoDfs = new SequentialFixpointRuleController(false);
+        SequentialFixpointRuleController seqCtrlFullDfs = new SequentialFixpointRuleController(true);
+        SequentialOnceRuleController seqOnceCtrl = new SequentialOnceRuleController(true);
+        DEFAULT_LOGICAL_REWRITES.add(new Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>(seqCtrlFullDfs,
+                HiveRuleCollections.NORMALIZATION));
+        DEFAULT_LOGICAL_REWRITES.add(new Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>(seqCtrlNoDfs,
+                HiveRuleCollections.COND_PUSHDOWN_AND_JOIN_INFERENCE));
+        DEFAULT_LOGICAL_REWRITES.add(new Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>(seqCtrlFullDfs,
+                HiveRuleCollections.LOAD_FIELDS));
+        DEFAULT_LOGICAL_REWRITES.add(new Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>(seqCtrlNoDfs,
+                HiveRuleCollections.OP_PUSHDOWN));
+        DEFAULT_LOGICAL_REWRITES.add(new Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>(seqOnceCtrl,
+                HiveRuleCollections.DATA_EXCHANGE));
+        DEFAULT_LOGICAL_REWRITES.add(new Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>(seqCtrlNoDfs,
+                HiveRuleCollections.CONSOLIDATION));
+
+        DEFAULT_PHYSICAL_REWRITES.add(new Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>(seqOnceCtrl,
+                HiveRuleCollections.PHYSICAL_PLAN_REWRITES));
+        DEFAULT_PHYSICAL_REWRITES.add(new Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>(seqOnceCtrl,
+                HiveRuleCollections.prepareJobGenRules));
+    }
+
+    /**
+     * static configurations for compiler
+     */
+    private HeuristicCompilerFactoryBuilder builder;
+
+    /**
+     * compiler
+     */
+    private ICompiler compiler;
+
+    /**
+     * physical optimization config
+     */
+    private PhysicalOptimizationConfig physicalOptimizationConfig;
+
+    /**
+     * final ending operators
+     */
+    private List<Operator> leaveOps = new ArrayList<Operator>();
+
+    /**
+     * tasks that are already visited
+     */
+    private Map<Task<? extends Serializable>, Boolean> tasksVisited = new HashMap<Task<? extends Serializable>, Boolean>();
+
+    /**
+     * hyracks job spec
+     */
+    private JobSpecification jobSpec;
+
+    /**
+     * hive configuration
+     */
+    private HiveConf conf;
+
+    /**
+     * plan printer
+     */
+    private PrintWriter planPrinter;
+
+    /**
+     * properties
+     */
+    private Properties clusterProps;
+
+    /**
+     * the Hyracks client connection
+     */
+    private IHyracksClientConnection hcc;
+
+    public HyracksExecutionEngine(HiveConf conf) {
+        this.conf = conf;
+        init(conf);
+    }
+
+    public HyracksExecutionEngine(HiveConf conf, PrintWriter planPrinter) {
+        this.conf = conf;
+        this.planPrinter = planPrinter;
+        init(conf);
+    }
+
+    private void init(HiveConf conf) {
+        builder = new HeuristicCompilerFactoryBuilder(DefaultOptimizationContextFactory.INSTANCE);
+        builder.setLogicalRewrites(DEFAULT_LOGICAL_REWRITES);
+        builder.setPhysicalRewrites(DEFAULT_PHYSICAL_REWRITES);
+        builder.setIMergeAggregationExpressionFactory(HiveMergeAggregationExpressionFactory.INSTANCE);
+        builder.setExpressionTypeComputer(HiveExpressionTypeComputer.INSTANCE);
+        builder.setNullableTypeComputer(HiveNullableTypeComputer.INSTANCE);
+
+        long memSizeExternalGby = conf.getLong("hive.algebricks.groupby.external.memory", 268435456);
+        long memSizeExternalSort = conf.getLong("hive.algebricks.sort.memory", 536870912);
+        int frameSize = conf.getInt("hive.algebricks.framesize", 32768);
+
+        physicalOptimizationConfig = new PhysicalOptimizationConfig();
+        int frameLimitExtGby = (int) (memSizeExternalGby / frameSize);
+        physicalOptimizationConfig.setMaxFramesExternalGroupBy(frameLimitExtGby);
+        int frameLimitExtSort = (int) (memSizeExternalSort / frameSize);
+        physicalOptimizationConfig.setMaxFramesExternalSort(frameLimitExtSort);
+        builder.setPhysicalOptimizationConfig(physicalOptimizationConfig);
+    }
+
+    @Override
+    public int compileJob(List<Task<? extends Serializable>> rootTasks) {
+        // clean up
+        leaveOps.clear();
+        tasksVisited.clear();
+        jobSpec = null;
+
+        HashMap<String, PartitionDesc> aliasToPath = new HashMap<String, PartitionDesc>();
+        List<Operator> rootOps = generateRootOperatorDAG(rootTasks, aliasToPath);
+
+        // get all leave Ops
+        getLeaves(rootOps, leaveOps);
+
+        HiveAlgebricksTranslator translator = new HiveAlgebricksTranslator();
+        try {
+            translator.translate(rootOps, null, aliasToPath);
+
+            ILogicalPlan plan = translator.genLogicalPlan();
+
+            if (plan.getRoots() != null && plan.getRoots().size() > 0 && plan.getRoots().get(0).getValue() != null) {
+                translator.printOperators();
+                ILogicalPlanAndMetadata planAndMetadata = new HiveLogicalPlanAndMetaData(plan,
+                        translator.getMetadataProvider());
+
+                ICompilerFactory compilerFactory = builder.create();
+                compiler = compilerFactory.createCompiler(planAndMetadata.getPlan(),
+                        planAndMetadata.getMetadataProvider(), translator.getVariableCounter());
+
+                // run optimization and re-writing rules for Hive plan
+                compiler.optimize();
+
+                // print optimized plan
+                LogicalOperatorPrettyPrintVisitor pvisitor = new LogicalOperatorPrettyPrintVisitor();
+                StringBuilder buffer = new StringBuilder();
+                PlanPrettyPrinter.printPlan(plan, buffer, pvisitor, 0);
+                String planStr = buffer.toString();
+                System.out.println(planStr);
+
+                if (planPrinter != null)
+                    planPrinter.print(planStr);
+            } else {
+                /** it is not a map reduce task DAG */
+                return 2;
+            }
+        } catch (Exception e) {
+            e.printStackTrace();
+            return 1;
+        }
+
+        return 0;
+    }
+
+    private void codeGen() throws AlgebricksException {
+        try {
+            // number of cpu cores in the cluster
+            builder.setClusterLocations(new AlgebricksAbsolutePartitionConstraint(ConfUtil.getNCs()));
+        } catch (Exception e) {
+            throw new AlgebricksException(e);
+        }
+        // builder.setClusterTopology(ConfUtil.getClusterTopology());
+        builder.setBinaryBooleanInspectorFactory(HiveBinaryBooleanInspectorFactory.INSTANCE);
+        builder.setBinaryIntegerInspectorFactory(HiveBinaryIntegerInspectorFactory.INSTANCE);
+        builder.setComparatorFactoryProvider(HiveBinaryComparatorFactoryProvider.INSTANCE);
+        builder.setExpressionRuntimeProvider(HiveExpressionRuntimeProvider.INSTANCE);
+        builder.setHashFunctionFactoryProvider(HiveBinaryHashFunctionFactoryProvider.INSTANCE);
+        builder.setPrinterProvider(HivePrinterFactoryProvider.INSTANCE);
+        builder.setSerializerDeserializerProvider(HiveSerializerDeserializerProvider.INSTANCE);
+        builder.setNullWriterFactory(HiveNullWriterFactory.INSTANCE);
+        builder.setNormalizedKeyComputerFactoryProvider(HiveNormalizedKeyComputerFactoryProvider.INSTANCE);
+        builder.setPartialAggregationTypeComputer(HivePartialAggregationTypeComputer.INSTANCE);
+        builder.setTypeTraitProvider(HiveTypeTraitProvider.INSTANCE);
+        builder.setHashFunctionFamilyProvider(HiveBinaryHashFunctionFamilyProvider.INSTANCE);
+
+        jobSpec = compiler.createJob(null, null);
+
+        // set the policy
+        String policyStr = conf.get("hive.hyracks.connectorpolicy");
+        if (policyStr == null)
+            policyStr = "PIPELINING";
+        Policy policyValue = Policy.valueOf(policyStr);
+        jobSpec.setConnectorPolicyAssignmentPolicy(new HiveConnectorPolicyAssignmentPolicy(policyValue));
+        jobSpec.setUseConnectorPolicyForScheduling(false);
+    }
+
+    @Override
+    public int executeJob() {
+        try {
+            codeGen();
+            executeHyracksJob(jobSpec);
+        } catch (Exception e) {
+            e.printStackTrace();
+            return 1;
+        }
+        return 0;
+    }
+
+    private List<Operator> generateRootOperatorDAG(List<Task<? extends Serializable>> rootTasks,
+            HashMap<String, PartitionDesc> aliasToPath) {
+
+        List<Operator> rootOps = new ArrayList<Operator>();
+        List<Task<? extends Serializable>> toDelete = new ArrayList<Task<? extends Serializable>>();
+        tasksVisited.clear();
+
+        for (int i = rootTasks.size() - 1; i >= 0; i--) {
+            /**
+             * list of map-reduce tasks
+             */
+            Task<? extends Serializable> task = rootTasks.get(i);
+
+            if (task instanceof MapRedTask) {
+                List<Operator> mapRootOps = articulateMapReduceOperators(task, rootOps, aliasToPath, rootTasks);
+                if (i == 0)
+                    rootOps.addAll(mapRootOps);
+                else {
+                    List<Operator> leaves = new ArrayList<Operator>();
+                    getLeaves(rootOps, leaves);
+
+                    List<Operator> mapChildren = new ArrayList<Operator>();
+                    for (Operator childMap : mapRootOps) {
+                        if (childMap instanceof TableScanOperator) {
+                            TableScanDesc topDesc = (TableScanDesc) childMap.getConf();
+                            if (topDesc == null)
+                                mapChildren.add(childMap);
+                            else {
+                                rootOps.add(childMap);
+                            }
+                        } else
+                            mapChildren.add(childMap);
+                    }
+
+                    if (mapChildren.size() > 0) {
+                        for (Operator leaf : leaves)
+                            leaf.setChildOperators(mapChildren);
+                        for (Operator child : mapChildren)
+                            child.setParentOperators(leaves);
+                    }
+                }
+
+                MapredWork mr = (MapredWork) task.getWork();
+                HashMap<String, PartitionDesc> map = mr.getAliasToPartnInfo();
+
+                addAliasToPartition(aliasToPath, map);
+                toDelete.add(task);
+            }
+        }
+
+        for (Task<? extends Serializable> task : toDelete)
+            rootTasks.remove(task);
+
+        return rootOps;
+    }
+
+    private void addAliasToPartition(HashMap<String, PartitionDesc> aliasToPath, HashMap<String, PartitionDesc> map) {
+        Iterator<String> keys = map.keySet().iterator();
+        while (keys.hasNext()) {
+            String key = keys.next();
+            PartitionDesc part = map.get(key);
+            String[] names = key.split(":");
+            for (String name : names) {
+                aliasToPath.put(name, part);
+            }
+        }
+    }
+
+    private List<Operator> articulateMapReduceOperators(Task task, List<Operator> rootOps,
+            HashMap<String, PartitionDesc> aliasToPath, List<Task<? extends Serializable>> rootTasks) {
+        // System.out.println("!"+task.getName());
+        if (!(task instanceof MapRedTask)) {
+            if (!(task instanceof ConditionalTask)) {
+                rootTasks.add(task);
+                return null;
+            } else {
+                // remove map-reduce branches in condition task
+                ConditionalTask condition = (ConditionalTask) task;
+                List<Task<? extends Serializable>> branches = condition.getListTasks();
+                for (int i = branches.size() - 1; i >= 0; i--) {
+                    Task branch = branches.get(i);
+                    if (branch instanceof MapRedTask) {
+                        return articulateMapReduceOperators(branch, rootOps, aliasToPath, rootTasks);
+                    }
+                }
+                rootTasks.add(task);
+                return null;
+            }
+        }
+
+        MapredWork mr = (MapredWork) task.getWork();
+        HashMap<String, PartitionDesc> map = mr.getAliasToPartnInfo();
+
+        // put all aliasToParitionDesc mapping into the map
+        addAliasToPartition(aliasToPath, map);
+
+        MapRedTask mrtask = (MapRedTask) task;
+        MapredWork work = (MapredWork) mrtask.getWork();
+        HashMap<String, Operator<? extends Serializable>> operators = work.getAliasToWork();
+
+        Set entries = operators.entrySet();
+        Iterator<Entry<String, Operator>> iterator = entries.iterator();
+        List<Operator> mapRootOps = new ArrayList<Operator>();
+
+        // get map root operators
+        while (iterator.hasNext()) {
+            Operator next = iterator.next().getValue();
+            if (!mapRootOps.contains(next)) {
+                // clear that only for the case of union
+                mapRootOps.add(next);
+            }
+        }
+
+        // get map local work
+        MapredLocalWork localWork = work.getMapLocalWork();
+        if (localWork != null) {
+            HashMap<String, Operator<? extends Serializable>> localOperators = localWork.getAliasToWork();
+
+            Set localEntries = localOperators.entrySet();
+            Iterator<Entry<String, Operator>> localIterator = localEntries.iterator();
+            while (localIterator.hasNext()) {
+                mapRootOps.add(localIterator.next().getValue());
+            }
+
+            HashMap<String, FetchWork> localFetch = localWork.getAliasToFetchWork();
+            Set localFetchEntries = localFetch.entrySet();
+            Iterator<Entry<String, FetchWork>> localFetchIterator = localFetchEntries.iterator();
+            while (localFetchIterator.hasNext()) {
+                Entry<String, FetchWork> fetchMap = localFetchIterator.next();
+                FetchWork fetch = fetchMap.getValue();
+                String alias = fetchMap.getKey();
+                List<PartitionDesc> dirPart = fetch.getPartDesc();
+
+                // temporary hack: put the first partitionDesc into the map
+                aliasToPath.put(alias, dirPart.get(0));
+            }
+        }
+
+        Boolean visited = tasksVisited.get(task);
+        if (visited != null && visited.booleanValue() == true) {
+            return mapRootOps;
+        }
+
+        // do that only for union operator
+        for (Operator op : mapRootOps)
+            if (op.getParentOperators() != null)
+                op.getParentOperators().clear();
+
+        List<Operator> mapLeaves = new ArrayList<Operator>();
+        downToLeaves(mapRootOps, mapLeaves);
+        List<Operator> reduceOps = new ArrayList<Operator>();
+
+        if (work.getReducer() != null)
+            reduceOps.add(work.getReducer());
+
+        for (Operator mapLeaf : mapLeaves) {
+            mapLeaf.setChildOperators(reduceOps);
+        }
+
+        for (Operator reduceOp : reduceOps) {
+            if (reduceOp != null)
+                reduceOp.setParentOperators(mapLeaves);
+        }
+
+        List<Operator> leafs = new ArrayList<Operator>();
+        if (reduceOps.size() > 0) {
+            downToLeaves(reduceOps, leafs);
+        } else {
+            leafs = mapLeaves;
+        }
+
+        List<Operator> mapChildren = new ArrayList<Operator>();
+        if (task.getChildTasks() != null && task.getChildTasks().size() > 0) {
+            for (Object child : task.getChildTasks()) {
+                List<Operator> childMapOps = articulateMapReduceOperators((Task) child, rootOps, aliasToPath, rootTasks);
+                if (childMapOps == null)
+                    continue;
+
+                for (Operator childMap : childMapOps) {
+                    if (childMap instanceof TableScanOperator) {
+                        TableScanDesc topDesc = (TableScanDesc) childMap.getConf();
+                        if (topDesc == null)
+                            mapChildren.add(childMap);
+                        else {
+                            rootOps.add(childMap);
+                        }
+                    } else {
+                        // if not table scan, add the child
+                        mapChildren.add(childMap);
+                    }
+                }
+            }
+
+            if (mapChildren.size() > 0) {
+                int i = 0;
+                for (Operator leaf : leafs) {
+                    if (leaf.getChildOperators() == null || leaf.getChildOperators().size() == 0)
+                        leaf.setChildOperators(new ArrayList<Operator>());
+                    leaf.getChildOperators().add(mapChildren.get(i));
+                    i++;
+                }
+                i = 0;
+                for (Operator child : mapChildren) {
+                    if (child.getParentOperators() == null || child.getParentOperators().size() == 0)
+                        child.setParentOperators(new ArrayList<Operator>());
+                    child.getParentOperators().add(leafs.get(i));
+                    i++;
+                }
+            }
+        }
+
+        // mark this task as visited
+        this.tasksVisited.put(task, true);
+        return mapRootOps;
+    }
+
+    /**
+     * down to leaf nodes
+     * 
+     * @param ops
+     * @param leaves
+     */
+    private void downToLeaves(List<Operator> ops, List<Operator> leaves) {
+
+        // Operator currentOp;
+        for (Operator op : ops) {
+            if (op != null && op.getChildOperators() != null && op.getChildOperators().size() > 0) {
+                downToLeaves(op.getChildOperators(), leaves);
+            } else {
+                if (op != null && leaves.indexOf(op) < 0)
+                    leaves.add(op);
+            }
+        }
+    }
+
+    private void getLeaves(List<Operator> roots, List<Operator> currentLeaves) {
+        for (Operator op : roots) {
+            List<Operator> children = op.getChildOperators();
+            if (children == null || children.size() <= 0) {
+                currentLeaves.add(op);
+            } else {
+                getLeaves(children, currentLeaves);
+            }
+        }
+    }
+
+    private void executeHyracksJob(JobSpecification job) throws Exception {
+
+        /**
+         * load the properties file if it is not loaded
+         */
+        if (clusterProps == null) {
+            clusterProps = new Properties();
+            InputStream confIn = new FileInputStream(clusterPropertiesPath);
+            clusterProps.load(confIn);
+            confIn.close();
+        }
+
+        if (hcc == null) {
+            BufferedReader ipReader = new BufferedReader(new InputStreamReader(new FileInputStream(masterFilePath)));
+            String masterNode = ipReader.readLine();
+            ipReader.close();
+
+            InetAddress[] ips = InetAddress.getAllByName(masterNode);
+            int port = Integer.parseInt(clusterProps.getProperty("CC_CLIENTPORT"));
+            for (InetAddress ip : ips) {
+                if (ip.getAddress().length <= 4) {
+                    try {
+                        hcc = new HyracksConnection(ip.getHostAddress(), port);
+                        break;
+                    } catch (Exception e) {
+                        continue;
+                    }
+                }
+            }
+        }
+
+        long start = System.currentTimeMillis();
+        JobId jobId = hcc.startJob(job);
+        hcc.waitForCompletion(jobId);
+
+        // System.out.println("job finished: " + jobId.toString());
+        // call all leave nodes to end
+        for (Operator leaf : leaveOps) {
+            jobClose(leaf);
+        }
+
+        long end = System.currentTimeMillis();
+        System.err.println(start + " " + end + " " + (end - start));
+    }
+
+    /**
+     * mv to final directory on hdfs (not real final)
+     * 
+     * @param leaf
+     * @throws Exception
+     */
+    private void jobClose(Operator leaf) throws Exception {
+        FileSinkOperator fsOp = (FileSinkOperator) leaf;
+        FileSinkDesc desc = fsOp.getConf();
+        boolean isNativeTable = !desc.getTableInfo().isNonNative();
+        if ((conf != null) && isNativeTable) {
+            String specPath = desc.getDirName();
+            DynamicPartitionCtx dpCtx = desc.getDynPartCtx();
+            // for 0.7.0
+            fsOp.mvFileToFinalPath(specPath, conf, true, LOG, dpCtx);
+            // for 0.8.0
+            // Utilities.mvFileToFinalPath(specPath, conf, true, LOG, dpCtx,
+            // desc);
+        }
+    }
+}
diff --git a/hivesterix/hivesterix-dist/src/main/java/org/apache/hadoop/hive/ql/Driver.java b/hivesterix/hivesterix-dist/src/main/java/org/apache/hadoop/hive/ql/Driver.java
index 4c40f5d..4ef74e9 100644
--- a/hivesterix/hivesterix-dist/src/main/java/org/apache/hadoop/hive/ql/Driver.java
+++ b/hivesterix/hivesterix-dist/src/main/java/org/apache/hadoop/hive/ql/Driver.java
@@ -444,10 +444,11 @@
 
             // hyracks run
             if (sem instanceof SemanticAnalyzer && command.toLowerCase().indexOf("create") < 0) {
-                hivesterix = true;
-                return engine.compileJob(sem.getRootTasks());
+                int engineRet = engine.compileJob(sem.getRootTasks());
+                if (engineRet == 0) {
+                    hivesterix = true;
+                }
             }
-
             return 0;
         } catch (SemanticException e) {
             errorMessage = "FAILED: Error in semantic analysis: " + e.getMessage();
diff --git a/hivesterix/hivesterix-dist/src/main/resources/conf/hive-default.xml b/hivesterix/hivesterix-dist/src/main/resources/conf/hive-default.xml
index 1fce28e..23a842a 100644
--- a/hivesterix/hivesterix-dist/src/main/resources/conf/hive-default.xml
+++ b/hivesterix/hivesterix-dist/src/main/resources/conf/hive-default.xml
@@ -36,7 +36,8 @@
 			whereas hive uses -1 as its default value.
 			By setting this property to -1, Hive will automatically figure out what
 			should be the number of reducers.
-  </description>
+        	</description>
+        </property>
 
         <property>
 		<name>hive.hyracks.connectorpolicy</name>
diff --git a/hivesterix/hivesterix-dist/src/main/resources/scripts/getip.sh b/hivesterix/hivesterix-dist/src/main/resources/scripts/getip.sh
index d11b091..0e737c5 100755
--- a/hivesterix/hivesterix-dist/src/main/resources/scripts/getip.sh
+++ b/hivesterix/hivesterix-dist/src/main/resources/scripts/getip.sh
@@ -47,10 +47,15 @@
 		IPADDR=`/sbin/ifconfig lo | grep "inet " | awk '{print $2}' | cut -f 2 -d ':'`
         fi 
 else
-        IPADDR=`/sbin/ifconfig en1 | grep "inet " | awk '{print $2}' | cut -f 2 -d ':'`
-	if [ "$IPADDR" = "" ]
+        #Get IP Address
+        IPADDR=`/sbin/ifconfig en0 | grep "inet " | awk '{print $2}' | cut -f 2 -d ':'`
+        if [ "$IPADDR" = "" ]
         then
-                IPADDR=`/sbin/ifconfig lo0 | grep "inet " | awk '{print $2}' | cut -f 2 -d ':'`
+                IPADDR=`/sbin/ifconfig en1 | grep "inet " | awk '{print $2}' | cut -f 2 -d ':'`
+        fi
+        if [ "$IPADDR" = "" ]
+        then
+                IPADDR=`/sbin/ifconfig lo | grep "inet " | awk '{print $2}' | cut -f 2 -d ':'`
         fi
 
 fi
diff --git a/hivesterix/hivesterix-runtime/pom.xml b/hivesterix/hivesterix-runtime/pom.xml
index 01db115..e4f5416 100644
--- a/hivesterix/hivesterix-runtime/pom.xml
+++ b/hivesterix/hivesterix-runtime/pom.xml
@@ -14,9 +14,7 @@
  !-->
 <project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
 	<modelVersion>4.0.0</modelVersion>
-	<groupId>edu.uci.ics.hyracks</groupId>
 	<artifactId>hivesterix-runtime</artifactId>
-	<version>0.2.7-SNAPSHOT</version>
 	<name>hivesterix-runtime</name>
 
 	<parent>
diff --git a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/application/INCApplicationContext.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/application/INCApplicationContext.java
index 11a0558..3d70a40 100644
--- a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/application/INCApplicationContext.java
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/application/INCApplicationContext.java
@@ -15,12 +15,12 @@
 package edu.uci.ics.hyracks.api.application;
 
 import edu.uci.ics.hyracks.api.context.IHyracksRootContext;
+import edu.uci.ics.hyracks.api.resources.memory.IMemoryManager;
 
 /**
  * Application Context at the Node Controller for an application.
  * 
  * @author vinayakb
- * 
  */
 public interface INCApplicationContext extends IApplicationContext {
     /**
@@ -38,8 +38,7 @@
     public IHyracksRootContext getRootContext();
 
     /**
-     * Set an object that can be later retrieved by the
-     * {@link #getApplicationObject()} call.
+     * Set an object that can be later retrieved by the {@link #getApplicationObject()} call.
      * 
      * @param object
      *            Application Object
@@ -47,10 +46,16 @@
     public void setApplicationObject(Object object);
 
     /**
-     * Get the application object previously set by the
-     * {@link #setApplicationObject(Object)} call.
+     * Get the application object previously set by the {@link #setApplicationObject(Object)} call.
      * 
      * @return Application Object
      */
     public Object getApplicationObject();
+
+    /**
+     * Get the memory manager at the node.
+     * 
+     * @return Memory Manager
+     */
+    public IMemoryManager getMemoryManager();
 }
\ No newline at end of file
diff --git a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/context/IHyracksCommonContext.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/context/IHyracksCommonContext.java
index f5d7a90..c75f9f9 100644
--- a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/context/IHyracksCommonContext.java
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/context/IHyracksCommonContext.java
@@ -16,6 +16,7 @@
 
 import java.nio.ByteBuffer;
 
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.api.io.IIOManager;
 
 public interface IHyracksCommonContext {
@@ -23,5 +24,5 @@
 
     public IIOManager getIOManager();
 
-    public ByteBuffer allocateFrame();
+    public ByteBuffer allocateFrame() throws HyracksDataException;
 }
\ No newline at end of file
diff --git a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/IOperatorNodePushable.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/IOperatorNodePushable.java
index acdda02..0e3fefd 100644
--- a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/IOperatorNodePushable.java
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/IOperatorNodePushable.java
@@ -25,7 +25,8 @@
 
     public int getInputArity();
 
-    public void setOutputFrameWriter(int index, IFrameWriter writer, RecordDescriptor recordDesc);
+    public void setOutputFrameWriter(int index, IFrameWriter writer, RecordDescriptor recordDesc)
+            throws HyracksDataException;
 
     public IFrameWriter getInputFrameWriter(int index);
 
diff --git a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/resources/memory/IMemoryManager.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/resources/memory/IMemoryManager.java
new file mode 100644
index 0000000..1017725
--- /dev/null
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/resources/memory/IMemoryManager.java
@@ -0,0 +1,11 @@
+package edu.uci.ics.hyracks.api.resources.memory;
+
+public interface IMemoryManager {
+    public long getMaximumMemory();
+
+    public long getAvailableMemory();
+
+    public boolean allocate(long memory);
+
+    public void deallocate(long memory);
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/rewriter/runtime/SuperActivityOperatorNodePushable.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/rewriter/runtime/SuperActivityOperatorNodePushable.java
index 7d50fa0..ffed337 100644
--- a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/rewriter/runtime/SuperActivityOperatorNodePushable.java
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/rewriter/runtime/SuperActivityOperatorNodePushable.java
@@ -167,7 +167,8 @@
     }
 
     @Override
-    public void setOutputFrameWriter(int clusterOutputIndex, IFrameWriter writer, RecordDescriptor recordDesc) {
+    public void setOutputFrameWriter(int clusterOutputIndex, IFrameWriter writer, RecordDescriptor recordDesc)
+            throws HyracksDataException {
         /**
          * set the right output frame writer
          */
diff --git a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/Joblet.java b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/Joblet.java
index 3395d69..56b6654 100644
--- a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/Joblet.java
+++ b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/Joblet.java
@@ -20,6 +20,8 @@
 import java.util.HashMap;
 import java.util.Hashtable;
 import java.util.Map;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.logging.Logger;
 
 import edu.uci.ics.hyracks.api.application.INCApplicationContext;
 import edu.uci.ics.hyracks.api.comm.IPartitionCollector;
@@ -56,6 +58,8 @@
 import edu.uci.ics.hyracks.control.nc.resources.DefaultDeallocatableRegistry;
 
 public class Joblet implements IHyracksJobletContext, ICounterContext {
+    private static final Logger LOGGER = Logger.getLogger(Joblet.class.getName());
+
     private final NodeControllerService nodeController;
 
     private final INCApplicationContext appCtx;
@@ -86,6 +90,8 @@
 
     private final int frameSize;
 
+    private final AtomicLong memoryAllocation;
+
     private JobStatus cleanupStatus;
 
     private boolean cleanupPending;
@@ -97,6 +103,7 @@
         this.deploymentId = deploymentId;
         this.jobId = jobId;
         this.frameSize = acg.getFrameSize();
+        memoryAllocation = new AtomicLong();
         this.acg = acg;
         partitionRequestMap = new HashMap<PartitionId, IPartitionCollector>();
         env = new OperatorEnvironmentImpl(nodeController.getId());
@@ -201,6 +208,11 @@
     }
 
     public void close() {
+        long stillAllocated = memoryAllocation.get();
+        if (stillAllocated > 0) {
+            LOGGER.warning("Freeing leaked " + stillAllocated + " bytes");
+            appCtx.getMemoryManager().deallocate(stillAllocated);
+        }
         nodeController.getExecutor().execute(new Runnable() {
             @Override
             public void run() {
@@ -209,11 +221,20 @@
         });
     }
 
-    ByteBuffer allocateFrame() {
-        return ByteBuffer.allocate(getFrameSize());
+    ByteBuffer allocateFrame() throws HyracksDataException {
+        if (appCtx.getMemoryManager().allocate(frameSize)) {
+            memoryAllocation.addAndGet(frameSize);
+            return ByteBuffer.allocate(frameSize);
+        }
+        throw new HyracksDataException("Unable to allocate frame: Not enough memory");
     }
 
-    int getFrameSize() {
+    public void deallocateFrames(int nFrames) {
+        memoryAllocation.addAndGet(nFrames * frameSize);
+        appCtx.getMemoryManager().deallocate(nFrames * frameSize);
+    }
+
+    final int getFrameSize() {
         return frameSize;
     }
 
diff --git a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/NodeControllerService.java b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/NodeControllerService.java
index 60859e3..6049a3b 100644
--- a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/NodeControllerService.java
+++ b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/NodeControllerService.java
@@ -69,6 +69,7 @@
 import edu.uci.ics.hyracks.control.nc.net.DatasetNetworkManager;
 import edu.uci.ics.hyracks.control.nc.net.NetworkManager;
 import edu.uci.ics.hyracks.control.nc.partitions.PartitionManager;
+import edu.uci.ics.hyracks.control.nc.resources.memory.MemoryManager;
 import edu.uci.ics.hyracks.control.nc.runtime.RootHyracksContext;
 import edu.uci.ics.hyracks.control.nc.work.AbortTasksWork;
 import edu.uci.ics.hyracks.control.nc.work.ApplicationMessageWork;
@@ -87,6 +88,8 @@
 public class NodeControllerService extends AbstractRemoteService {
     private static Logger LOGGER = Logger.getLogger(NodeControllerService.class.getName());
 
+    private static final double MEMORY_FUDGE_FACTOR = 0.8;
+
     private NCConfig ncConfig;
 
     private final String id;
@@ -139,6 +142,8 @@
 
     private final Mutable<FutureValue<Map<String, NodeControllerInfo>>> getNodeControllerInfosAcceptor;
 
+    private final MemoryManager memoryManager;
+
     public NodeControllerService(NCConfig ncConfig) throws Exception {
         this.ncConfig = ncConfig;
         id = ncConfig.nodeId;
@@ -165,6 +170,7 @@
         osMXBean = ManagementFactory.getOperatingSystemMXBean();
         registrationPending = true;
         getNodeControllerInfosAcceptor = new MutableObject<FutureValue<Map<String, NodeControllerInfo>>>();
+        memoryManager = new MemoryManager((long) (memoryMXBean.getHeapMemoryUsage().getMax() * MEMORY_FUDGE_FACTOR));
     }
 
     public IHyracksRootContext getRootContext() {
@@ -275,7 +281,7 @@
     }
 
     private void startApplication() throws Exception {
-        appCtx = new NCApplicationContext(serverCtx, ctx, id);
+        appCtx = new NCApplicationContext(serverCtx, ctx, id, memoryManager);
         String className = ncConfig.appNCMainClass;
         if (className != null) {
             Class<?> c = Class.forName(className);
diff --git a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/Task.java b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/Task.java
index 1c92354..53e5a01 100644
--- a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/Task.java
+++ b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/Task.java
@@ -109,7 +109,7 @@
     }
 
     @Override
-    public ByteBuffer allocateFrame() {
+    public ByteBuffer allocateFrame() throws HyracksDataException {
         return joblet.allocateFrame();
     }
 
diff --git a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/application/NCApplicationContext.java b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/application/NCApplicationContext.java
index 8a1f0d9..4b8eb53 100644
--- a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/application/NCApplicationContext.java
+++ b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/application/NCApplicationContext.java
@@ -19,18 +19,23 @@
 
 import edu.uci.ics.hyracks.api.application.INCApplicationContext;
 import edu.uci.ics.hyracks.api.context.IHyracksRootContext;
+import edu.uci.ics.hyracks.api.resources.memory.IMemoryManager;
 import edu.uci.ics.hyracks.control.common.application.ApplicationContext;
 import edu.uci.ics.hyracks.control.common.context.ServerContext;
+import edu.uci.ics.hyracks.control.nc.resources.memory.MemoryManager;
 
 public class NCApplicationContext extends ApplicationContext implements INCApplicationContext {
     private final String nodeId;
     private final IHyracksRootContext rootCtx;
+    private final MemoryManager memoryManager;
     private Object appObject;
 
-    public NCApplicationContext(ServerContext serverCtx, IHyracksRootContext rootCtx, String nodeId) throws IOException {
+    public NCApplicationContext(ServerContext serverCtx, IHyracksRootContext rootCtx, String nodeId,
+            MemoryManager memoryManager) throws IOException {
         super(serverCtx);
         this.nodeId = nodeId;
         this.rootCtx = rootCtx;
+        this.memoryManager = memoryManager;
     }
 
     @Override
@@ -56,4 +61,9 @@
     public Object getApplicationObject() {
         return appObject;
     }
+
+    @Override
+    public IMemoryManager getMemoryManager() {
+        return memoryManager;
+    }
 }
\ No newline at end of file
diff --git a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/resources/memory/MemoryManager.java b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/resources/memory/MemoryManager.java
new file mode 100644
index 0000000..8add260
--- /dev/null
+++ b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/resources/memory/MemoryManager.java
@@ -0,0 +1,40 @@
+package edu.uci.ics.hyracks.control.nc.resources.memory;
+
+import java.util.concurrent.atomic.AtomicLong;
+
+import edu.uci.ics.hyracks.api.resources.memory.IMemoryManager;
+
+public class MemoryManager implements IMemoryManager {
+    private final long maxMemory;
+
+    private final AtomicLong memory;
+
+    public MemoryManager(long maxMemory) {
+        this.maxMemory = maxMemory;
+        this.memory = new AtomicLong(maxMemory);
+    }
+
+    @Override
+    public long getMaximumMemory() {
+        return maxMemory;
+    }
+
+    @Override
+    public long getAvailableMemory() {
+        return memory.get();
+    }
+
+    @Override
+    public boolean allocate(long memory) {
+        if (this.memory.addAndGet(-memory) < 0) {
+            this.memory.addAndGet(memory);
+            return false;
+        }
+        return true;
+    }
+
+    @Override
+    public void deallocate(long memory) {
+        this.memory.addAndGet(memory);
+    }
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/FrameDeserializingDataReader.java b/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/FrameDeserializingDataReader.java
index 424af4c..b94a219 100644
--- a/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/FrameDeserializingDataReader.java
+++ b/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/FrameDeserializingDataReader.java
@@ -34,7 +34,7 @@
     private final FrameDeserializer frameDeserializer;
 
     public FrameDeserializingDataReader(IHyracksTaskContext ctx, IFrameReader frameReader,
-            RecordDescriptor recordDescriptor) {
+            RecordDescriptor recordDescriptor) throws HyracksDataException {
         buffer = ctx.allocateFrame();
         this.frameReader = frameReader;
         this.frameDeserializer = new FrameDeserializer(ctx.getFrameSize(), recordDescriptor);
diff --git a/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/SerializingDataWriter.java b/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/SerializingDataWriter.java
index 3d23c31..4392fd6 100644
--- a/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/SerializingDataWriter.java
+++ b/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/SerializingDataWriter.java
@@ -39,7 +39,8 @@
 
     private boolean open;
 
-    public SerializingDataWriter(IHyracksTaskContext ctx, RecordDescriptor recordDescriptor, IFrameWriter frameWriter) {
+    public SerializingDataWriter(IHyracksTaskContext ctx, RecordDescriptor recordDescriptor, IFrameWriter frameWriter)
+            throws HyracksDataException {
         buffer = ctx.allocateFrame();
         tb = new ArrayTupleBuilder(recordDescriptor.getFieldCount());
         this.recordDescriptor = recordDescriptor;
diff --git a/hyracks/hyracks-dataflow-hadoop/src/main/java/edu/uci/ics/hyracks/dataflow/hadoop/mapreduce/ReduceWriter.java b/hyracks/hyracks-dataflow-hadoop/src/main/java/edu/uci/ics/hyracks/dataflow/hadoop/mapreduce/ReduceWriter.java
index 5fbd2a5..f84ffda3 100644
--- a/hyracks/hyracks-dataflow-hadoop/src/main/java/edu/uci/ics/hyracks/dataflow/hadoop/mapreduce/ReduceWriter.java
+++ b/hyracks/hyracks-dataflow-hadoop/src/main/java/edu/uci/ics/hyracks/dataflow/hadoop/mapreduce/ReduceWriter.java
@@ -60,7 +60,7 @@
     public ReduceWriter(IHyracksTaskContext ctx, HadoopHelper helper, int[] groupFields,
             IBinaryComparatorFactory[] comparatorFactories, RecordDescriptor recordDescriptor,
             Reducer<K2, V2, K3, V3> reducer, RecordWriter<K3, V3> recordWriter, TaskAttemptID taId,
-            TaskAttemptContext taskAttemptContext) {
+            TaskAttemptContext taskAttemptContext) throws HyracksDataException {
         this.ctx = ctx;
         this.helper = helper;
         this.groupFields = groupFields;
diff --git a/hyracks/hyracks-dataflow-hadoop/src/main/java/edu/uci/ics/hyracks/dataflow/hadoop/mapreduce/ShuffleFrameReader.java b/hyracks/hyracks-dataflow-hadoop/src/main/java/edu/uci/ics/hyracks/dataflow/hadoop/mapreduce/ShuffleFrameReader.java
index b34fa9f..30ba3ec 100644
--- a/hyracks/hyracks-dataflow-hadoop/src/main/java/edu/uci/ics/hyracks/dataflow/hadoop/mapreduce/ShuffleFrameReader.java
+++ b/hyracks/hyracks-dataflow-hadoop/src/main/java/edu/uci/ics/hyracks/dataflow/hadoop/mapreduce/ShuffleFrameReader.java
@@ -134,7 +134,7 @@
         private RunFileWriter rfw;
         private int blockId;
 
-        public RunInfo() {
+        public RunInfo() throws HyracksDataException {
             buffer = ctx.allocateFrame();
             fta = new FrameTupleAppender(ctx.getFrameSize());
         }
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/FileScanOperatorDescriptor.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/FileScanOperatorDescriptor.java
index 82a7797..d36b29e 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/FileScanOperatorDescriptor.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/FileScanOperatorDescriptor.java
@@ -45,7 +45,7 @@
 
     @Override
     public IOperatorNodePushable createPushRuntime(IHyracksTaskContext ctx,
-            IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) {
+            IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) throws HyracksDataException {
         final FileSplit split = fileSplitProvider.getFileSplits()[partition];
         final ITupleParser tp = tupleParserFactory.createTupleParser(ctx);
         return new AbstractUnaryOutputSourceOperatorNodePushable() {
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/ITupleParserFactory.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/ITupleParserFactory.java
index 7393072..e2518ee 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/ITupleParserFactory.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/ITupleParserFactory.java
@@ -17,7 +17,8 @@
 import java.io.Serializable;
 
 import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 
 public interface ITupleParserFactory extends Serializable {
-    public ITupleParser createTupleParser(IHyracksTaskContext ctx);
+    public ITupleParser createTupleParser(IHyracksTaskContext ctx) throws HyracksDataException;
 }
\ No newline at end of file
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/HashSpillableTableFactory.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/HashSpillableTableFactory.java
index 8cf929e..ecd5284 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/HashSpillableTableFactory.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/HashSpillableTableFactory.java
@@ -366,8 +366,9 @@
              * When frames are already created, they are recycled.
              * 
              * @return Whether a new frame is added successfully.
+             * @throws HyracksDataException
              */
-            private boolean nextAvailableFrame() {
+            private boolean nextAvailableFrame() throws HyracksDataException {
                 // Return false if the number of frames is equal to the limit.
                 if (lastBufIndex + 1 >= framesLimit)
                     return false;
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/hash/GroupingHashTable.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/hash/GroupingHashTable.java
index f11b12a..f8ba003 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/hash/GroupingHashTable.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/hash/GroupingHashTable.java
@@ -142,7 +142,7 @@
         outputTupleBuilder = new ArrayTupleBuilder(outRecordDescriptor.getFields().length);
     }
 
-    private void addNewBuffer() {
+    private void addNewBuffer() throws HyracksDataException {
         ByteBuffer buffer = ctx.allocateFrame();
         buffer.position(0);
         buffer.limit(buffer.capacity());
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/preclustered/PreclusteredGroupWriter.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/preclustered/PreclusteredGroupWriter.java
index 11458ad..7773765 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/preclustered/PreclusteredGroupWriter.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/preclustered/PreclusteredGroupWriter.java
@@ -46,7 +46,7 @@
 
     public PreclusteredGroupWriter(IHyracksTaskContext ctx, int[] groupFields, IBinaryComparator[] comparators,
             IAggregatorDescriptor aggregator, RecordDescriptor inRecordDesc, RecordDescriptor outRecordDesc,
-            IFrameWriter writer) {
+            IFrameWriter writer) throws HyracksDataException {
         this.groupFields = groupFields;
         this.comparators = comparators;
         this.aggregator = aggregator;
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/HybridHashJoinOperatorDescriptor.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/HybridHashJoinOperatorDescriptor.java
index 37fa40b..1bfa413 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/HybridHashJoinOperatorDescriptor.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/HybridHashJoinOperatorDescriptor.java
@@ -181,7 +181,7 @@
 
         @Override
         public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx,
-                IRecordDescriptorProvider recordDescProvider, final int partition, final int nPartitions) {
+                IRecordDescriptorProvider recordDescProvider, final int partition, final int nPartitions) throws HyracksDataException {
             final RecordDescriptor rd0 = recordDescProvider.getInputRecordDescriptor(joinAid, 0);
             final RecordDescriptor rd1 = recordDescProvider.getInputRecordDescriptor(getActivityId(), 0);
             final IBinaryComparator[] comparators = new IBinaryComparator[comparatorFactories.length];
@@ -370,7 +370,8 @@
 
         @Override
         public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx,
-                IRecordDescriptorProvider recordDescProvider, final int partition, final int nPartitions) {
+                IRecordDescriptorProvider recordDescProvider, final int partition, final int nPartitions)
+                throws HyracksDataException {
             final RecordDescriptor rd0 = recordDescProvider.getInputRecordDescriptor(getActivityId(), 0);
             final RecordDescriptor rd1 = recordDescProvider.getInputRecordDescriptor(buildAid, 0);
             final IBinaryComparator[] comparators = new IBinaryComparator[comparatorFactories.length];
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/NestedLoopJoin.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/NestedLoopJoin.java
index 7585698..2f719fa 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/NestedLoopJoin.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/NestedLoopJoin.java
@@ -109,7 +109,7 @@
         reloadFrame(outerBuffer);
     }
 
-    private void createAndCopyFrame(ByteBuffer outerBuffer) {
+    private void createAndCopyFrame(ByteBuffer outerBuffer) throws HyracksDataException {
         ByteBuffer outerBufferCopy = ctx.allocateFrame();
         FrameUtils.copy(outerBuffer, outerBufferCopy);
         outBuffers.add(outerBufferCopy);
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/OptimizedHybridHashJoin.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/OptimizedHybridHashJoin.java
index 47a9a1c..cd32c81 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/OptimizedHybridHashJoin.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/OptimizedHybridHashJoin.java
@@ -39,8 +39,8 @@
 
 /**
  * @author pouria
-       This class mainly applies one level of HHJ on a pair of
-       relations. It is always called by the descriptor.
+ *         This class mainly applies one level of HHJ on a pair of
+ *         relations. It is always called by the descriptor.
  */
 public class OptimizedHybridHashJoin {
 
@@ -49,7 +49,7 @@
     private final int INVALID_BUFFER = -2;
     private final int UNALLOCATED_FRAME = -3;
     private final int BUFFER_FOR_RESIDENT_PARTS = -1;
-    
+
     private IHyracksTaskContext ctx;
 
     private final String rel0Name;
@@ -107,10 +107,10 @@
         this.memForJoin = memForJoin;
         this.buildRd = buildRd;
         this.probeRd = probeRd;
-        this.buildHpc = buildHpc; 	
-        this.probeHpc = probeHpc; 	
-        this.buildKeys = keys1; 	
-        this.probeKeys = keys0;		
+        this.buildHpc = buildHpc;
+        this.probeHpc = probeHpc;
+        this.buildKeys = keys1;
+        this.probeKeys = keys0;
         this.comparators = comparators;
         this.rel0Name = rel0Name;
         this.rel1Name = rel1Name;
@@ -136,10 +136,10 @@
         this.memForJoin = memForJoin;
         this.buildRd = buildRd;
         this.probeRd = probeRd;
-        this.buildHpc = buildHpc; 	
-        this.probeHpc = probeHpc; 	
-        this.buildKeys = keys1; 	
-        this.probeKeys = keys0;		
+        this.buildHpc = buildHpc;
+        this.probeHpc = probeHpc;
+        this.buildKeys = keys1;
+        this.probeKeys = keys0;
         this.comparators = comparators;
         this.rel0Name = rel0Name;
         this.rel1Name = rel1Name;
@@ -162,7 +162,7 @@
         }
     }
 
-    public void initBuild() {
+    public void initBuild() throws HyracksDataException {
         memBuffs = new ByteBuffer[memForJoin];
         curPBuff = new int[numOfPartitions];
         nextBuff = new int[memForJoin];
@@ -191,12 +191,12 @@
     public void build(ByteBuffer buffer) throws HyracksDataException {
         accessorBuild.reset(buffer);
         int tupleCount = accessorBuild.getTupleCount();
-   
+
         boolean print = false;
-    	if(print){
-    		accessorBuild.prettyPrint();
-    	}
-        
+        if (print) {
+            accessorBuild.prettyPrint();
+        }
+
         for (int i = 0; i < tupleCount; ++i) {
             int pid = buildHpc.partition(accessorBuild, i, numOfPartitions);
             processTuple(i, pid);
@@ -225,7 +225,7 @@
                     buildTupAppender.reset(memBuffs[pidToSpill], true);
                     processTuple(tid, pid);
                     break;
-                }  //New Buffer allocated successfully
+                } //New Buffer allocated successfully
                 partition = memBuffs[curPBuff[pid]]; //Current Buffer for the partition is now updated by allocateFreeBuffer() call above
                 buildTupAppender.reset(partition, true);
                 if (!buildTupAppender.append(accessorBuild, tid)) {
@@ -250,7 +250,7 @@
         }
     }
 
-    private int allocateFreeBuffer(int pid) {
+    private int allocateFreeBuffer(int pid) throws HyracksDataException {
         if (nextFreeBuffIx != NO_MORE_FREE_BUFFER) {
             if (memBuffs[nextFreeBuffIx] == null) {
                 memBuffs[nextFreeBuffIx] = ctx.allocateFrame();
@@ -425,7 +425,7 @@
     private ArrayList<Integer> selectPartitionsToReload() {
         ArrayList<Integer> p = new ArrayList<Integer>();
         for (int i = pStatus.nextSetBit(0); i >= 0; i = pStatus.nextSetBit(i + 1)) {
-            if (buildPSizeInFrames[i]>0 && (freeFramesCounter - buildPSizeInFrames[i] >= 0) ) {
+            if (buildPSizeInFrames[i] > 0 && (freeFramesCounter - buildPSizeInFrames[i] >= 0)) {
                 p.add(i);
                 freeFramesCounter -= buildPSizeInFrames[i];
             }
@@ -457,7 +457,7 @@
         }
     }
 
-    public void initProbe() {
+    public void initProbe() throws HyracksDataException {
 
         sPartBuffs = new ByteBuffer[numOfSpilledParts];
         for (int i = 0; i < numOfSpilledParts; i++) {
@@ -468,7 +468,7 @@
         /* We only need to allocate one frame per spilled partition. 
          * Resident partitions do not need frames in probe, as their tuples join 
          * immediately with the resident build tuples using the inMemoryHashJoin */
-        for (int i = 0; i < numOfPartitions; i++) { 
+        for (int i = 0; i < numOfPartitions; i++) {
             curPBuff[i] = (pStatus.get(i)) ? nextBuffIxToAlloc++ : BUFFER_FOR_RESIDENT_PARTS;
         }
         probePSizeInTups = new int[numOfPartitions];
@@ -488,10 +488,10 @@
         int tupleCount = accessorProbe.getTupleCount();
 
         boolean print = false;
-    	if(print){
-    		accessorProbe.prettyPrint();
-    	}
-        
+        if (print) {
+            accessorProbe.prettyPrint();
+        }
+
         if (numOfSpilledParts == 0) {
             inMemJoiner.join(buffer, writer);
             return;
@@ -508,14 +508,14 @@
                         probeTupAppenderToSpilled.reset(buff, needToClear);
                         if (probeTupAppenderToSpilled.append(accessorProbe, i)) {
                             break;
-                        } 
+                        }
                         probeWrite(pid, buff);
                         buff.clear();
                         needToClear = true;
                     }
                 } else { //pid is Resident
                     while (true) {
-                        if (probeTupAppenderToResident.append(accessorProbe, i)){
+                        if (probeTupAppenderToResident.append(accessorProbe, i)) {
                             break;
                         }
                         inMemJoiner.join(probeResBuff, writer);
@@ -635,8 +635,8 @@
                 + freeFramesCounter;
         return s;
     }
-    
-    public boolean isTableEmpty(){
-    	return this.isTableEmpty;
+
+    public boolean isTableEmpty() {
+        return this.isTableEmpty;
     }
 }
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/OptimizedHybridHashJoinOperatorDescriptor.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/OptimizedHybridHashJoinOperatorDescriptor.java
index 35d2468..95b7a3c 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/OptimizedHybridHashJoinOperatorDescriptor.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/OptimizedHybridHashJoinOperatorDescriptor.java
@@ -335,7 +335,8 @@
 
         @Override
         public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx,
-                IRecordDescriptorProvider recordDescProvider, final int partition, final int nPartitions) {
+                IRecordDescriptorProvider recordDescProvider, final int partition, final int nPartitions)
+                throws HyracksDataException {
 
             final RecordDescriptor probeRd = recordDescProvider.getInputRecordDescriptor(buildAid, 0);
             final RecordDescriptor buildRd = recordDescProvider.getInputRecordDescriptor(getActivityId(), 0);
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/result/ResultWriterOperatorDescriptor.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/result/ResultWriterOperatorDescriptor.java
index 7330590..42ed59e 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/result/ResultWriterOperatorDescriptor.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/result/ResultWriterOperatorDescriptor.java
@@ -57,7 +57,8 @@
 
     @Override
     public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx,
-            IRecordDescriptorProvider recordDescProvider, final int partition, final int nPartitions) {
+            IRecordDescriptorProvider recordDescProvider, final int partition, final int nPartitions)
+            throws HyracksDataException {
         final IDatasetPartitionManager dpm = ctx.getDatasetPartitionManager();
 
         final ByteBuffer outputBuffer = ctx.allocateFrame();
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/BSTMemMgr.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/BSTMemMgr.java
index d1d0d5c..110bddb 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/BSTMemMgr.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/BSTMemMgr.java
@@ -632,7 +632,7 @@
         BSTNodeUtil.setPrev(frameIx, offset, BSTNodeUtil.INVALID_INDEX, BSTNodeUtil.INVALID_INDEX, frames);
     }
 
-    private ByteBuffer allocateFrame() {
+    private ByteBuffer allocateFrame() throws HyracksDataException {
         return ctx.allocateFrame();
     }
 
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/ExternalSortRunGenerator.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/ExternalSortRunGenerator.java
index 6e297a7..b149e30 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/ExternalSortRunGenerator.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/ExternalSortRunGenerator.java
@@ -36,7 +36,7 @@
 
     public ExternalSortRunGenerator(IHyracksTaskContext ctx, int[] sortFields,
             INormalizedKeyComputerFactory firstKeyNormalizerFactory, IBinaryComparatorFactory[] comparatorFactories,
-            RecordDescriptor recordDesc, int framesLimit) {
+            RecordDescriptor recordDesc, int framesLimit) throws HyracksDataException {
         this.ctx = ctx;
         frameSorter = new FrameSorter(ctx, sortFields, firstKeyNormalizerFactory, comparatorFactories, recordDesc);
         runs = new LinkedList<IFrameReader>();
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/FrameSorter.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/FrameSorter.java
index cb7b311..db70a62 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/FrameSorter.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/FrameSorter.java
@@ -50,7 +50,7 @@
 
     public FrameSorter(IHyracksTaskContext ctx, int[] sortFields,
             INormalizedKeyComputerFactory firstKeyNormalizerFactory, IBinaryComparatorFactory[] comparatorFactories,
-            RecordDescriptor recordDescriptor) {
+            RecordDescriptor recordDescriptor) throws HyracksDataException {
         this.ctx = ctx;
         this.sortFields = sortFields;
         nkc = firstKeyNormalizerFactory == null ? null : firstKeyNormalizerFactory.createNormalizedKeyComputer();
@@ -76,7 +76,7 @@
         return dataFrameCount;
     }
 
-    public void insertFrame(ByteBuffer buffer) {
+    public void insertFrame(ByteBuffer buffer) throws HyracksDataException {
         ByteBuffer copyFrame;
         if (dataFrameCount == buffers.size()) {
             copyFrame = ctx.allocateFrame();
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/structures/ISerializableTable.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/structures/ISerializableTable.java
index 7cbf41b..ce6c00c 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/structures/ISerializableTable.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/structures/ISerializableTable.java
@@ -14,9 +14,11 @@
  */
 package edu.uci.ics.hyracks.dataflow.std.structures;
 
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
 public interface ISerializableTable {
 
-    public void insert(int entry, TuplePointer tuplePointer);
+    public void insert(int entry, TuplePointer tuplePointer) throws HyracksDataException;
 
     public void getTuplePointer(int entry, int offset, TuplePointer tuplePointer);
 
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/structures/SerializableHashTable.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/structures/SerializableHashTable.java
index 54dd7a7..a245c9d 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/structures/SerializableHashTable.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/structures/SerializableHashTable.java
@@ -18,6 +18,7 @@
 import java.util.List;
 
 import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 
 /**
  * An entry in the table is: #elements, #no-empty elements; fIndex, tIndex;
@@ -38,7 +39,7 @@
     private int headerFrameCount = 0;
     private TuplePointer tempTuplePointer = new TuplePointer();
 
-    public SerializableHashTable(int tableSize, final IHyracksTaskContext ctx) {
+    public SerializableHashTable(int tableSize, final IHyracksTaskContext ctx) throws HyracksDataException {
         this.ctx = ctx;
         int frameSize = ctx.getFrameSize();
 
@@ -53,7 +54,7 @@
     }
 
     @Override
-    public void insert(int entry, TuplePointer pointer) {
+    public void insert(int entry, TuplePointer pointer) throws HyracksDataException {
         int hFrameIndex = getHeaderFrameIndex(entry);
         int headerOffset = getHeaderFrameOffset(entry);
         IntSerDeBuffer header = headers[hFrameIndex];
@@ -143,7 +144,8 @@
         currentLargestFrameIndex = 0;
     }
 
-    private void insertNewEntry(IntSerDeBuffer header, int headerOffset, int entryCapacity, TuplePointer pointer) {
+    private void insertNewEntry(IntSerDeBuffer header, int headerOffset, int entryCapacity, TuplePointer pointer)
+            throws HyracksDataException {
         IntSerDeBuffer lastFrame = contents.get(currentLargestFrameIndex);
         int lastIndex = frameCurrentIndex.get(currentLargestFrameIndex);
         int requiredIntCapacity = entryCapacity * 2;
@@ -191,7 +193,7 @@
     }
 
     private void insertNonFirstTuple(IntSerDeBuffer header, int headerOffset, int frameIndex, int offsetIndex,
-            TuplePointer pointer) {
+            TuplePointer pointer) throws HyracksDataException {
         IntSerDeBuffer frame = contents.get(frameIndex);
         int entryItems = frame.getInt(offsetIndex);
         int entryUsedItems = frame.getInt(offsetIndex + 1);
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/util/DeserializedOperatorNodePushable.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/util/DeserializedOperatorNodePushable.java
index c89bfba..aff4273 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/util/DeserializedOperatorNodePushable.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/util/DeserializedOperatorNodePushable.java
@@ -40,7 +40,8 @@
     }
 
     @Override
-    public void setOutputFrameWriter(int index, IFrameWriter writer, RecordDescriptor recordDesc) {
+    public void setOutputFrameWriter(int index, IFrameWriter writer, RecordDescriptor recordDesc)
+            throws HyracksDataException {
         delegate.setDataWriter(index, new SerializingDataWriter(ctx, recordDesc, writer));
     }
 
diff --git a/hyracks/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/PrimaryIndexBulkLoadExample.java b/hyracks/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/PrimaryIndexBulkLoadExample.java
index 1431d0f..a027570 100644
--- a/hyracks/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/PrimaryIndexBulkLoadExample.java
+++ b/hyracks/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/PrimaryIndexBulkLoadExample.java
@@ -149,7 +149,8 @@
         IIndexDataflowHelperFactory dataflowHelperFactory = new BTreeDataflowHelperFactory();
         TreeIndexBulkLoadOperatorDescriptor btreeBulkLoad = new TreeIndexBulkLoadOperatorDescriptor(spec,
                 storageManager, lcManagerProvider, btreeSplitProvider, typeTraits, comparatorFactories, null,
-                fieldPermutation, 0.7f, false, 1000L, dataflowHelperFactory, NoOpOperationCallbackFactory.INSTANCE);
+                fieldPermutation, 0.7f, false, 1000L, true, dataflowHelperFactory,
+                NoOpOperationCallbackFactory.INSTANCE);
         JobHelper.createPartitionConstraint(spec, btreeBulkLoad, splitNCs);
 
         // distribute the records from the datagen via hashing to the bulk load
diff --git a/hyracks/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/SecondaryIndexBulkLoadExample.java b/hyracks/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/SecondaryIndexBulkLoadExample.java
index b484f85..805660f 100644
--- a/hyracks/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/SecondaryIndexBulkLoadExample.java
+++ b/hyracks/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/SecondaryIndexBulkLoadExample.java
@@ -141,7 +141,8 @@
         IFileSplitProvider btreeSplitProvider = JobHelper.createFileSplitProvider(splitNCs, options.secondaryBTreeName);
         TreeIndexBulkLoadOperatorDescriptor btreeBulkLoad = new TreeIndexBulkLoadOperatorDescriptor(spec,
                 storageManager, lcManagerProvider, btreeSplitProvider, secondaryTypeTraits, comparatorFactories, null,
-                fieldPermutation, 0.7f, false, 1000L, dataflowHelperFactory, NoOpOperationCallbackFactory.INSTANCE);
+                fieldPermutation, 0.7f, false, 1000L, true, dataflowHelperFactory,
+                NoOpOperationCallbackFactory.INSTANCE);
         JobHelper.createPartitionConstraint(spec, btreeBulkLoad, splitNCs);
 
         // connect the ops
diff --git a/hyracks/hyracks-examples/btree-example/btreehelper/src/main/java/edu/uci/ics/hyracks/examples/btree/helper/DataGenOperatorDescriptor.java b/hyracks/hyracks-examples/btree-example/btreehelper/src/main/java/edu/uci/ics/hyracks/examples/btree/helper/DataGenOperatorDescriptor.java
index 4ac68e7..ed4cf0c 100644
--- a/hyracks/hyracks-examples/btree-example/btreehelper/src/main/java/edu/uci/ics/hyracks/examples/btree/helper/DataGenOperatorDescriptor.java
+++ b/hyracks/hyracks-examples/btree-example/btreehelper/src/main/java/edu/uci/ics/hyracks/examples/btree/helper/DataGenOperatorDescriptor.java
@@ -59,7 +59,7 @@
 
     @Override
     public IOperatorNodePushable createPushRuntime(IHyracksTaskContext ctx,
-            IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) {
+            IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) throws HyracksDataException {
 
         final ByteBuffer outputFrame = ctx.allocateFrame();
         final FrameTupleAppender appender = new FrameTupleAppender(ctx.getFrameSize());
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/btree/AbstractBTreeOperatorTest.java b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/btree/AbstractBTreeOperatorTest.java
index a93c199..eec7225 100644
--- a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/btree/AbstractBTreeOperatorTest.java
+++ b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/btree/AbstractBTreeOperatorTest.java
@@ -178,7 +178,7 @@
         int[] fieldPermutation = { 0, 1, 2, 4, 5, 7 };
         TreeIndexBulkLoadOperatorDescriptor primaryBtreeBulkLoad = new TreeIndexBulkLoadOperatorDescriptor(spec,
                 storageManager, lcManagerProvider, primarySplitProvider, primaryTypeTraits, primaryComparatorFactories,
-                primaryBloomFilterKeyFields, fieldPermutation, 0.7f, true, 1000L, dataflowHelperFactory,
+                primaryBloomFilterKeyFields, fieldPermutation, 0.7f, true, 1000L, true, dataflowHelperFactory,
                 NoOpOperationCallbackFactory.INSTANCE);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, primaryBtreeBulkLoad, NC1_ID);
 
@@ -241,7 +241,7 @@
         int[] fieldPermutation = { 3, 0 };
         TreeIndexBulkLoadOperatorDescriptor secondaryBtreeBulkLoad = new TreeIndexBulkLoadOperatorDescriptor(spec,
                 storageManager, lcManagerProvider, secondarySplitProvider, secondaryTypeTraits,
-                secondaryComparatorFactories, secondaryBloomFilterKeyFields, fieldPermutation, 0.7f, true, 1000L,
+                secondaryComparatorFactories, secondaryBloomFilterKeyFields, fieldPermutation, 0.7f, true, 1000L, true,
                 dataflowHelperFactory, NoOpOperationCallbackFactory.INSTANCE);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, secondaryBtreeBulkLoad, NC1_ID);
 
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/invertedindex/AbstractfWordInvertedIndexTest.java b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/invertedindex/AbstractfWordInvertedIndexTest.java
index 8bb698a..1bd22d7 100644
--- a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/invertedindex/AbstractfWordInvertedIndexTest.java
+++ b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/invertedindex/AbstractfWordInvertedIndexTest.java
@@ -203,8 +203,8 @@
         int[] fieldPermutation = { 0, 1 };
         TreeIndexBulkLoadOperatorDescriptor primaryBtreeBulkLoad = new TreeIndexBulkLoadOperatorDescriptor(spec,
                 storageManager, lcManagerProvider, primaryFileSplitProvider, primaryTypeTraits,
-                primaryComparatorFactories, null, fieldPermutation, 0.7f, true, 1000L, btreeDataflowHelperFactory,
-                NoOpOperationCallbackFactory.INSTANCE);
+                primaryComparatorFactories, null, fieldPermutation, 0.7f, true, 1000L, true,
+                btreeDataflowHelperFactory, NoOpOperationCallbackFactory.INSTANCE);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, primaryBtreeBulkLoad, NC1_ID);
         return primaryBtreeBulkLoad;
     }
@@ -277,7 +277,7 @@
 
     private IOperatorDescriptor createInvertedIndexBulkLoadOp(JobSpecification spec, int[] fieldPermutation) {
         LSMInvertedIndexBulkLoadOperatorDescriptor invIndexBulkLoadOp = new LSMInvertedIndexBulkLoadOperatorDescriptor(
-                spec, fieldPermutation, true, 1000L, storageManager, btreeFileSplitProvider, lcManagerProvider,
+                spec, fieldPermutation, true, 1000L, true, storageManager, btreeFileSplitProvider, lcManagerProvider,
                 tokenTypeTraits, tokenComparatorFactories, invListsTypeTraits, invListsComparatorFactories,
                 tokenizerFactory, invertedIndexDataflowHelperFactory, NoOpOperationCallbackFactory.INSTANCE);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, invIndexBulkLoadOp, NC1_ID);
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/rtree/AbstractRTreeOperatorTest.java b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/rtree/AbstractRTreeOperatorTest.java
index 00296a5..d1c682c 100644
--- a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/rtree/AbstractRTreeOperatorTest.java
+++ b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/rtree/AbstractRTreeOperatorTest.java
@@ -223,7 +223,7 @@
         int[] fieldPermutation = { 0, 1, 2, 4, 5, 7, 9, 10, 11, 12 };
         TreeIndexBulkLoadOperatorDescriptor primaryBulkLoad = new TreeIndexBulkLoadOperatorDescriptor(spec,
                 storageManager, lcManagerProvider, primarySplitProvider, primaryTypeTraits, primaryComparatorFactories,
-                null, fieldPermutation, 0.7f, false, 1000L, btreeDataflowHelperFactory,
+                null, fieldPermutation, 0.7f, false, 1000L, true, btreeDataflowHelperFactory,
                 NoOpOperationCallbackFactory.INSTANCE);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, primaryBulkLoad, NC1_ID);
 
@@ -280,8 +280,8 @@
         int[] fieldPermutation = { 6, 7, 8, 9, 0 };
         TreeIndexBulkLoadOperatorDescriptor secondaryBulkLoad = new TreeIndexBulkLoadOperatorDescriptor(spec,
                 storageManager, lcManagerProvider, secondarySplitProvider, secondaryTypeTraits,
-                secondaryComparatorFactories, null, fieldPermutation, 0.7f, false, 1000L, rtreeDataflowHelperFactory,
-                NoOpOperationCallbackFactory.INSTANCE);
+                secondaryComparatorFactories, null, fieldPermutation, 0.7f, false, 1000L, true,
+                rtreeDataflowHelperFactory, NoOpOperationCallbackFactory.INSTANCE);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, secondaryBulkLoad, NC1_ID);
 
         spec.connect(new OneToOneConnectorDescriptor(spec), keyProviderOp, 0, primarySearchOp, 0);
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/comm/SerializationDeserializationTest.java b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/comm/SerializationDeserializationTest.java
index c6efca4..29be04b 100644
--- a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/comm/SerializationDeserializationTest.java
+++ b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/comm/SerializationDeserializationTest.java
@@ -56,7 +56,7 @@
             buffers = new ArrayList<ByteBuffer>();
         }
 
-        public IOpenableDataWriter<Object[]> createWriter() {
+        public IOpenableDataWriter<Object[]> createWriter() throws HyracksDataException {
             return new SerializingDataWriter(ctx, rDes, new IFrameWriter() {
                 @Override
                 public void open() throws HyracksDataException {
@@ -79,7 +79,7 @@
             });
         }
 
-        public IOpenableDataReader<Object[]> createDataReader() {
+        public IOpenableDataReader<Object[]> createDataReader() throws HyracksDataException {
             return new FrameDeserializingDataReader(ctx, new IFrameReader() {
                 private int i;
 
diff --git a/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs/lib/TextKeyValueParserFactory.java b/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs/lib/TextKeyValueParserFactory.java
index 71a7680..fbac95b 100644
--- a/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs/lib/TextKeyValueParserFactory.java
+++ b/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs/lib/TextKeyValueParserFactory.java
@@ -33,7 +33,8 @@
     private static final long serialVersionUID = 1L;
 
     @Override
-    public IKeyValueParser<LongWritable, Text> createKeyValueParser(final IHyracksTaskContext ctx) {
+    public IKeyValueParser<LongWritable, Text> createKeyValueParser(final IHyracksTaskContext ctx)
+            throws HyracksDataException {
 
         final ArrayTupleBuilder tb = new ArrayTupleBuilder(1);
         final ByteBuffer buffer = ctx.allocateFrame();
diff --git a/hyracks/hyracks-storage-am-bloomfilter/src/main/java/edu/uci/ics/hyracks/storage/am/bloomfilter/impls/BloomFilter.java b/hyracks/hyracks-storage-am-bloomfilter/src/main/java/edu/uci/ics/hyracks/storage/am/bloomfilter/impls/BloomFilter.java
index 0e796b0..6811e1b 100644
--- a/hyracks/hyracks-storage-am-bloomfilter/src/main/java/edu/uci/ics/hyracks/storage/am/bloomfilter/impls/BloomFilter.java
+++ b/hyracks/hyracks-storage-am-bloomfilter/src/main/java/edu/uci/ics/hyracks/storage/am/bloomfilter/impls/BloomFilter.java
@@ -209,15 +209,15 @@
                 throw new HyracksDataException("Failed to create the bloom filter builder since it is not activated.");
             }
 
-            this.numElements = numElements;
+            this.numElements = numElements == 0 ? 1 : numElements;
             this.numHashes = numHashes;
-            numBits = numElements * numBitsPerElement;
+            numBits = this.numElements * numBitsPerElement;
             long tmp = (long) Math.ceil(numBits / (double) numBitsPerPage);
             if (tmp > Integer.MAX_VALUE) {
                 throw new HyracksDataException("Cannot create a bloom filter with his huge number of pages.");
             }
             numPages = (int) tmp;
-            if (numElements > 0) {
+            if (this.numElements > 0) {
                 persistBloomFilterMetaData();
                 readBloomFilterMetaData();
                 int currentPageId = 1;
diff --git a/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/impls/BTree.java b/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/impls/BTree.java
index 7c29fc9..689843b 100644
--- a/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/impls/BTree.java
+++ b/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/impls/BTree.java
@@ -934,8 +934,8 @@
     }
 
     @Override
-    public IIndexBulkLoader createBulkLoader(float fillFactor, boolean verifyInput, long numElementsHint)
-            throws TreeIndexException {
+    public IIndexBulkLoader createBulkLoader(float fillFactor, boolean verifyInput, long numElementsHint,
+            boolean checkIfEmptyIndex) throws TreeIndexException {
         try {
             return new BTreeBulkLoader(fillFactor, verifyInput);
         } catch (HyracksDataException e) {
@@ -1025,7 +1025,8 @@
                 HyracksDataException {
             // New tuple should be strictly greater than last tuple.
             if (cmp.compare(tuple, prevTuple) <= 0) {
-                throw new UnsortedInputException("Input stream given to BTree bulk load is not sorted.");
+                throw new UnsortedInputException(
+                        "Input stream given to BTree bulk load is not sorted or has duplicates.");
             }
         }
 
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 074ae7c..4006cb7 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
@@ -91,9 +91,10 @@
      *            the callback to be used for modification operations
      * @param searchCallback
      *            the callback to be used for search operations
+     * @throws HyracksDataException
      */
     public IIndexAccessor createAccessor(IModificationOperationCallback modificationCallback,
-            ISearchOperationCallback searchCallback);
+            ISearchOperationCallback searchCallback) throws HyracksDataException;
 
     /**
      * Ensures that all pages (and tuples) of the index are logically consistent.
@@ -119,6 +120,6 @@
      * @param verifyInput
      * @throws IndexException
      */
-    public IIndexBulkLoader createBulkLoader(float fillFactor, boolean verifyInput, long numElementsHint)
-            throws IndexException;
+    public IIndexBulkLoader createBulkLoader(float fillFactor, boolean verifyInput, long numElementsHint,
+            boolean checkIfEmptyIndex) throws IndexException;
 }
diff --git a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/IIndexOperationContext.java b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/IIndexOperationContext.java
index 31fc627..974d91c 100644
--- a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/IIndexOperationContext.java
+++ b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/IIndexOperationContext.java
@@ -14,11 +14,12 @@
  */
 package edu.uci.ics.hyracks.storage.am.common.api;
 
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.storage.am.common.ophelpers.IndexOperation;
 
 public interface IIndexOperationContext {
-    void setOperation(IndexOperation newOp);
-    
+    void setOperation(IndexOperation newOp) throws HyracksDataException;
+
     IndexOperation getOperation();
 
     void reset();
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 0504a94..a0f9d8e 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
@@ -34,6 +34,7 @@
     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;
@@ -43,13 +44,14 @@
 
     public IndexBulkLoadOperatorNodePushable(IIndexOperatorDescriptor opDesc, IHyracksTaskContext ctx, int partition,
             int[] fieldPermutation, float fillFactor, boolean verifyInput, long numElementsHint,
-            IRecordDescriptorProvider recordDescProvider) {
+            boolean checkIfEmptyIndex, IRecordDescriptorProvider recordDescProvider) {
         this.opDesc = opDesc;
         this.ctx = ctx;
         this.indexHelper = opDesc.getIndexDataflowHelperFactory().createIndexDataflowHelper(opDesc, ctx, partition);
         this.fillFactor = fillFactor;
         this.verifyInput = verifyInput;
         this.numElementsHint = numElementsHint;
+        this.checkIfEmptyIndex = checkIfEmptyIndex;
         this.recDescProvider = recordDescProvider;
         tuple.setFieldPermutation(fieldPermutation);
     }
@@ -61,7 +63,7 @@
         indexHelper.open();
         index = indexHelper.getIndexInstance();
         try {
-            bulkLoader = index.createBulkLoader(fillFactor, verifyInput, numElementsHint);
+            bulkLoader = index.createBulkLoader(fillFactor, verifyInput, numElementsHint, checkIfEmptyIndex);
         } catch (Exception e) {
             indexHelper.close();
             throw new HyracksDataException(e);
diff --git a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/IndexDataflowHelper.java b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/IndexDataflowHelper.java
index 8c2d847..5e07e0c 100644
--- a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/IndexDataflowHelper.java
+++ b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/IndexDataflowHelper.java
@@ -75,7 +75,7 @@
             // any physical artifact that the LocalResourceRepository is managing (e.g. a file containing the resource ID). 
             // Once the index has been created, a new resource ID can be generated.
             if (resourceID != -1) {
-                localResourceRepository.deleteResourceByName(file.getFile().getPath(), ioDeviceId);
+                localResourceRepository.deleteResourceByName(file.getFile().getPath());
             }
             index.create();
             try {
@@ -83,9 +83,8 @@
                 resourceID = resourceIdFactory.createId();
                 ILocalResourceFactory localResourceFactory = opDesc.getLocalResourceFactoryProvider()
                         .getLocalResourceFactory();
-                localResourceRepository.insert(
-                        localResourceFactory.createLocalResource(resourceID, file.getFile().getPath(), partition),
-                        ioDeviceId);
+                localResourceRepository.insert(localResourceFactory.createLocalResource(resourceID, file.getFile()
+                        .getPath(), partition));
             } catch (IOException e) {
                 throw new HyracksDataException(e);
             }
@@ -127,7 +126,7 @@
             }
 
             if (resourceID != -1) {
-                localResourceRepository.deleteResourceByName(file.getFile().getPath(), ioDeviceId);
+                localResourceRepository.deleteResourceByName(file.getFile().getPath());
             }
             index.destroy();
         }
diff --git a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexBulkLoadOperatorDescriptor.java b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexBulkLoadOperatorDescriptor.java
index 09cdd68..a58e9e8 100644
--- a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexBulkLoadOperatorDescriptor.java
+++ b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexBulkLoadOperatorDescriptor.java
@@ -36,12 +36,13 @@
     private final float fillFactor;
     private final boolean verifyInput;
     private final long numElementsHint;
+    private final boolean checkIfEmptyIndex;
 
     public TreeIndexBulkLoadOperatorDescriptor(IOperatorDescriptorRegistry spec,
             IStorageManagerInterface storageManager, IIndexLifecycleManagerProvider lifecycleManagerProvider,
             IFileSplitProvider fileSplitProvider, ITypeTraits[] typeTraits,
             IBinaryComparatorFactory[] comparatorFactories, int[] bloomFilterKeyFields, int[] fieldPermutation,
-            float fillFactor, boolean verifyInput, long numElementsHint,
+            float fillFactor, boolean verifyInput, long numElementsHint, boolean checkIfEmptyIndex,
             IIndexDataflowHelperFactory dataflowHelperFactory,
             IModificationOperationCallbackFactory modificationOpCallbackFactory) {
         super(spec, 1, 0, null, storageManager, lifecycleManagerProvider, fileSplitProvider, typeTraits,
@@ -52,12 +53,13 @@
         this.fillFactor = fillFactor;
         this.verifyInput = verifyInput;
         this.numElementsHint = numElementsHint;
+        this.checkIfEmptyIndex = checkIfEmptyIndex;
     }
 
     @Override
     public IOperatorNodePushable createPushRuntime(IHyracksTaskContext ctx,
             IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) {
         return new IndexBulkLoadOperatorNodePushable(this, ctx, partition, fieldPermutation, fillFactor, verifyInput,
-                numElementsHint, recordDescProvider);
+                numElementsHint, checkIfEmptyIndex, recordDescProvider);
     }
 }
\ No newline at end of file
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 b573a32..19d40a0 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
@@ -333,8 +333,12 @@
     }

 

     public class TreeIndexInsertBulkLoader implements IIndexBulkLoader {

-        ITreeIndexAccessor accessor = (ITreeIndexAccessor) createAccessor(NoOpOperationCallback.INSTANCE,

-                NoOpOperationCallback.INSTANCE);

+        ITreeIndexAccessor accessor;

+

+        public TreeIndexInsertBulkLoader() throws HyracksDataException {

+            accessor = (ITreeIndexAccessor) createAccessor(NoOpOperationCallback.INSTANCE,

+                    NoOpOperationCallback.INSTANCE);

+        }

 

         @Override

         public void add(ITupleReference tuple) throws HyracksDataException {

@@ -356,4 +360,4 @@
     public long getMemoryAllocationSize() {

         return 0;

     }

-}

+}
diff --git a/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/dataflow/LSMBTreeDataflowHelper.java b/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/dataflow/LSMBTreeDataflowHelper.java
index d34906e..f217419 100644
--- a/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/dataflow/LSMBTreeDataflowHelper.java
+++ b/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/dataflow/LSMBTreeDataflowHelper.java
@@ -49,11 +49,10 @@
     @Override
     public ITreeIndex createIndexInstance() throws HyracksDataException {
         AbstractTreeIndexOperatorDescriptor treeOpDesc = (AbstractTreeIndexOperatorDescriptor) opDesc;
-        return LSMBTreeUtils.createLSMTree(virtualBufferCache, ctx.getIOManager(), file, opDesc.getStorageManager()
-                .getBufferCache(ctx), opDesc.getStorageManager().getFileMapProvider(ctx), treeOpDesc
-                .getTreeIndexTypeTraits(), treeOpDesc.getTreeIndexComparatorFactories(), treeOpDesc
-                .getTreeIndexBloomFilterKeyFields(), bloomFilterFalsePositiveRate, mergePolicy, opTrackerFactory
-                .getOperationTracker(ctx), ioScheduler, ioOpCallbackProvider, opDesc.getFileSplitProvider()
-                .getFileSplits()[partition].getIODeviceId());
+        return LSMBTreeUtils.createLSMTree(virtualBufferCache, file, opDesc.getStorageManager().getBufferCache(ctx),
+                opDesc.getStorageManager().getFileMapProvider(ctx), treeOpDesc.getTreeIndexTypeTraits(),
+                treeOpDesc.getTreeIndexComparatorFactories(), treeOpDesc.getTreeIndexBloomFilterKeyFields(),
+                bloomFilterFalsePositiveRate, mergePolicy, opTrackerFactory.getOperationTracker(ctx), ioScheduler,
+                ioOpCallbackProvider);
     }
 }
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 8161200..3557d73 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
@@ -211,6 +211,7 @@
 
         List<ILSMComponent> immutableComponents = componentsRef.get();
         mutableComponent.getBTree().clear();
+        mutableComponent.reset();
         for (ILSMComponent c : immutableComponents) {
             LSMBTreeImmutableComponent component = (LSMBTreeImmutableComponent) c;
             component.getBloomFilter().deactivate();
@@ -369,7 +370,7 @@
 
         LSMBTreeImmutableComponent component = createDiskComponent(componentFactory, flushOp.getBTreeFlushTarget(),
                 flushOp.getBloomFilterFlushTarget(), true);
-        IIndexBulkLoader bulkLoader = component.getBTree().createBulkLoader(1.0f, false, numElements);
+        IIndexBulkLoader bulkLoader = component.getBTree().createBulkLoader(1.0f, false, numElements, false);
         IIndexBulkLoader builder = component.getBloomFilter().createBuilder(numElements,
                 bloomFilterSpec.getNumHashes(), bloomFilterSpec.getNumBucketsPerElements());
 
@@ -429,7 +430,7 @@
         LSMBTreeImmutableComponent mergedComponent = createDiskComponent(componentFactory,
                 mergeOp.getBTreeMergeTarget(), mergeOp.getBloomFilterMergeTarget(), true);
 
-        IIndexBulkLoader bulkLoader = mergedComponent.getBTree().createBulkLoader(1.0f, false, numElements);
+        IIndexBulkLoader bulkLoader = mergedComponent.getBTree().createBulkLoader(1.0f, false, numElements, false);
         IIndexBulkLoader builder = mergedComponent.getBloomFilter().createBuilder(numElements,
                 bloomFilterSpec.getNumHashes(), bloomFilterSpec.getNumBucketsPerElements());
         try {
@@ -464,10 +465,10 @@
     }
 
     @Override
-    public IIndexBulkLoader createBulkLoader(float fillLevel, boolean verifyInput, long numElementsHint)
-            throws TreeIndexException {
+    public IIndexBulkLoader createBulkLoader(float fillLevel, boolean verifyInput, long numElementsHint,
+            boolean checkIfEmptyIndex) throws TreeIndexException {
         try {
-            return new LSMBTreeBulkLoader(fillLevel, verifyInput, numElementsHint);
+            return new LSMBTreeBulkLoader(fillLevel, verifyInput, numElementsHint, checkIfEmptyIndex);
         } catch (HyracksDataException e) {
             throw new TreeIndexException(e);
         }
@@ -498,19 +499,22 @@
         private final ILSMComponent component;
         private final BTreeBulkLoader bulkLoader;
         private final IIndexBulkLoader builder;
-        private boolean endHasBeenCalled = false;
+        private boolean cleanedUpArtifacts = false;
+        private boolean isEmptyComponent = true;
+        private boolean endedBloomFilterLoad = false;
 
-        public LSMBTreeBulkLoader(float fillFactor, boolean verifyInput, long numElementsHint)
+        public LSMBTreeBulkLoader(float fillFactor, boolean verifyInput, long numElementsHint, boolean checkIfEmptyIndex)
                 throws TreeIndexException, HyracksDataException {
+            if (checkIfEmptyIndex && !isEmptyIndex()) {
+                throw new TreeIndexException("Cannot load an index that is not empty");
+            }
             try {
                 component = createBulkLoadTarget();
-            } catch (HyracksDataException e) {
-                throw new TreeIndexException(e);
-            } catch (IndexException e) {
+            } catch (HyracksDataException | IndexException e) {
                 throw new TreeIndexException(e);
             }
             bulkLoader = (BTreeBulkLoader) ((LSMBTreeImmutableComponent) component).getBTree().createBulkLoader(
-                    fillFactor, verifyInput, numElementsHint);
+                    fillFactor, verifyInput, numElementsHint, false);
 
             int maxBucketsPerElement = BloomCalculations.maxBucketsPerElement(numElementsHint);
             BloomFilterSpecification bloomFilterSpec = BloomCalculations.computeBloomSpec(maxBucketsPerElement,
@@ -524,36 +528,45 @@
             try {
                 bulkLoader.add(tuple);
                 builder.add(tuple);
-            } catch (IndexException e) {
-                handleException();
+            } catch (IndexException | HyracksDataException | RuntimeException e) {
+                cleanupArtifacts();
                 throw e;
-            } catch (HyracksDataException e) {
-                handleException();
-                throw e;
-            } catch (RuntimeException e) {
-                handleException();
-                throw e;
+            }
+            if (isEmptyComponent) {
+                isEmptyComponent = false;
             }
         }
 
-        protected void handleException() throws HyracksDataException, IndexException {
-            if (!endHasBeenCalled) {
-                builder.end();
+        protected 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;
+                }
+                ((LSMBTreeImmutableComponent) component).getBTree().deactivate();
+                ((LSMBTreeImmutableComponent) component).getBTree().destroy();
+                ((LSMBTreeImmutableComponent) component).getBloomFilter().deactivate();
+                ((LSMBTreeImmutableComponent) component).getBloomFilter().destroy();
             }
-            ((LSMBTreeImmutableComponent) component).getBTree().deactivate();
-            ((LSMBTreeImmutableComponent) component).getBTree().destroy();
-            ((LSMBTreeImmutableComponent) component).getBloomFilter().deactivate();
-            ((LSMBTreeImmutableComponent) component).getBloomFilter().destroy();
         }
 
         @Override
         public void end() throws HyracksDataException, IndexException {
-            bulkLoader.end();
-            builder.end();
-            endHasBeenCalled = true;
-            lsmHarness.addBulkLoadedComponent(component);
+            if (!cleanedUpArtifacts) {
+                if (!endedBloomFilterLoad) {
+                    builder.end();
+                    endedBloomFilterLoad = true;
+                }
+                bulkLoader.end();
+                if (isEmptyComponent) {
+                    cleanupArtifacts();
+                } else {
+                    lsmHarness.addBulkLoadedComponent(component);
+                }
+            }
         }
-
     }
 
     public LSMBTreeOpContext createOpContext(IModificationOperationCallback modificationCallback,
@@ -630,10 +643,7 @@
     }
 
     public boolean isEmptyIndex() throws HyracksDataException {
-        List<ILSMComponent> immutableComponents = componentsRef.get();
-        return immutableComponents.isEmpty()
-                && mutableComponent.getBTree().isEmptyTree(
-                        mutableComponent.getBTree().getInteriorFrameFactory().createFrame());
+        return componentsRef.get().isEmpty() && !mutableComponent.isModified();
     }
 
     @Override
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 b71e9be..7e8fbb3 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
@@ -26,7 +26,6 @@
 
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.api.io.FileReference;
-import edu.uci.ics.hyracks.api.io.IIOManager;
 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;
@@ -39,9 +38,9 @@
 
     private final TreeIndexFactory<? extends ITreeIndex> btreeFactory;
 
-    public LSMBTreeFileManager(IIOManager ioManager, IFileMapProvider fileMapProvider, FileReference file,
-            TreeIndexFactory<? extends ITreeIndex> btreeFactory, int ioDeviceId) {
-        super(ioManager, fileMapProvider, file, null, ioDeviceId);
+    public LSMBTreeFileManager(IFileMapProvider fileMapProvider, FileReference file,
+            TreeIndexFactory<? extends ITreeIndex> btreeFactory) {
+        super(fileMapProvider, file, null);
         this.btreeFactory = btreeFactory;
     }
 
@@ -79,16 +78,16 @@
         ArrayList<ComparableFileName> allBTreeFiles = new ArrayList<ComparableFileName>();
         ArrayList<ComparableFileName> allBloomFilterFiles = new ArrayList<ComparableFileName>();
 
-        // Gather files from the IODeviceHandle.
+        // Gather files
 
         // List of valid BTree files.
-        cleanupAndGetValidFilesInternal(dev, btreeFilter, btreeFactory, allBTreeFiles);
+        cleanupAndGetValidFilesInternal(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(dev, btreeFilesSet, allBloomFilterFiles, bloomFilterFilter, null);
+        validateFiles(btreeFilesSet, allBloomFilterFiles, bloomFilterFilter, null);
 
         // Sanity check.
         if (allBTreeFiles.size() != allBloomFilterFiles.size()) {
diff --git a/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/impls/LSMBTreeOpContext.java b/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/impls/LSMBTreeOpContext.java
index 6a94150..ac62d9f 100644
--- a/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/impls/LSMBTreeOpContext.java
+++ b/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/impls/LSMBTreeOpContext.java
@@ -52,13 +52,12 @@
             ISearchOperationCallback searchCallback, int numBloomFilterKeyFields) {
         IBinaryComparatorFactory cmpFactories[] = memBTree.getComparatorFactories();
         if (cmpFactories[0] != null) {
-            this.cmp = MultiComparator.createIgnoreFieldLength(memBTree.getComparatorFactories());
+            this.cmp = MultiComparator.create(memBTree.getComparatorFactories());
         } else {
             this.cmp = null;
         }
 
-        bloomFilterCmp = MultiComparator.createIgnoreFieldLength(memBTree.getComparatorFactories(), 0,
-                numBloomFilterKeyFields);
+        bloomFilterCmp = MultiComparator.create(memBTree.getComparatorFactories(), 0, numBloomFilterKeyFields);
 
         this.memBTree = memBTree;
         this.insertLeafFrameFactory = insertLeafFrameFactory;
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 28bcb45..507ec02 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
@@ -18,7 +18,6 @@
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
 import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
 import edu.uci.ics.hyracks.api.io.FileReference;
-import edu.uci.ics.hyracks.api.io.IIOManager;
 import edu.uci.ics.hyracks.storage.am.bloomfilter.impls.BloomFilterFactory;
 import edu.uci.ics.hyracks.storage.am.btree.frames.BTreeNSMInteriorFrameFactory;
 import edu.uci.ics.hyracks.storage.am.btree.frames.BTreeNSMLeafFrameFactory;
@@ -44,22 +43,11 @@
 import edu.uci.ics.hyracks.storage.common.file.IFileMapProvider;
 
 public class LSMBTreeUtils {
-
-    public static LSMBTree createLSMTree(IVirtualBufferCache virtualBufferCache, IIOManager ioManager,
-            FileReference file, IBufferCache diskBufferCache, IFileMapProvider diskFileMapProvider,
-            ITypeTraits[] typeTraits, IBinaryComparatorFactory[] cmpFactories, int[] bloomFilterKeyFields,
-            double bloomFilterFalsePositiveRate, ILSMMergePolicy mergePolicy, ILSMOperationTracker opTracker,
-            ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallbackProvider ioOpCallbackProvider) {
-        return createLSMTree(virtualBufferCache, ioManager, file, diskBufferCache, diskFileMapProvider, typeTraits,
-                cmpFactories, bloomFilterKeyFields, bloomFilterFalsePositiveRate, mergePolicy, opTracker, ioScheduler,
-                ioOpCallbackProvider, 0);
-    }
-
-    public static LSMBTree createLSMTree(IVirtualBufferCache virtualBufferCache, IIOManager ioManager,
-            FileReference file, IBufferCache diskBufferCache, IFileMapProvider diskFileMapProvider,
-            ITypeTraits[] typeTraits, IBinaryComparatorFactory[] cmpFactories, int[] bloomFilterKeyFields,
-            double bloomFilterFalsePositiveRate, ILSMMergePolicy mergePolicy, ILSMOperationTracker opTracker,
-            ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallbackProvider ioOpCallbackProvider, int ioDeviceId) {
+    public static LSMBTree createLSMTree(IVirtualBufferCache virtualBufferCache, FileReference file,
+            IBufferCache diskBufferCache, IFileMapProvider diskFileMapProvider, ITypeTraits[] typeTraits,
+            IBinaryComparatorFactory[] cmpFactories, int[] bloomFilterKeyFields, double bloomFilterFalsePositiveRate,
+            ILSMMergePolicy mergePolicy, ILSMOperationTracker opTracker, ILSMIOOperationScheduler ioScheduler,
+            ILSMIOOperationCallbackProvider ioOpCallbackProvider) {
         LSMBTreeTupleWriterFactory insertTupleWriterFactory = new LSMBTreeTupleWriterFactory(typeTraits,
                 cmpFactories.length, false);
         LSMBTreeTupleWriterFactory deleteTupleWriterFactory = new LSMBTreeTupleWriterFactory(typeTraits,
@@ -83,8 +71,7 @@
         BloomFilterFactory bloomFilterFactory = new BloomFilterFactory(diskBufferCache, diskFileMapProvider,
                 bloomFilterKeyFields);
 
-        ILSMIndexFileManager fileNameManager = new LSMBTreeFileManager(ioManager, diskFileMapProvider, file,
-                diskBTreeFactory, ioDeviceId);
+        ILSMIndexFileManager fileNameManager = new LSMBTreeFileManager(diskFileMapProvider, file, diskBTreeFactory);
 
         LSMBTree lsmTree = new LSMBTree(virtualBufferCache, interiorFrameFactory, insertLeafFrameFactory,
                 deleteLeafFrameFactory, fileNameManager, diskBTreeFactory, bulkLoadBTreeFactory, bloomFilterFactory,
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 3f71c67..a6425ce 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
@@ -28,8 +28,6 @@
 
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.api.io.FileReference;
-import edu.uci.ics.hyracks.api.io.IIOManager;
-import edu.uci.ics.hyracks.api.io.IODeviceHandle;
 import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
 import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexMetaDataFrame;
 import edu.uci.ics.hyracks.storage.am.common.api.IndexException;
@@ -45,7 +43,6 @@
     protected static final String BLOOM_FILTER_STRING = "f";
 
     protected final IFileMapProvider fileMapProvider;
-    protected final IODeviceHandle dev;
 
     // baseDir should reflect dataset name and partition name.
     protected String baseDir;
@@ -55,15 +52,14 @@
 
     protected final TreeIndexFactory<? extends ITreeIndex> treeFactory;
 
-    public AbstractLSMIndexFileManager(IIOManager ioManager, IFileMapProvider fileMapProvider, FileReference file,
-            TreeIndexFactory<? extends ITreeIndex> treeFactory, int ioDeviceId) {
+    public AbstractLSMIndexFileManager(IFileMapProvider fileMapProvider, FileReference file,
+            TreeIndexFactory<? extends ITreeIndex> treeFactory) {
         this.baseDir = file.getFile().getPath();
         if (!baseDir.endsWith(System.getProperty("file.separator"))) {
             baseDir += System.getProperty("file.separator");
         }
         this.fileMapProvider = fileMapProvider;
         this.treeFactory = treeFactory;
-        this.dev = ioManager.getIODevices().get(ioDeviceId);
     }
 
     private static FilenameFilter fileNameFilter = new FilenameFilter() {
@@ -94,10 +90,10 @@
         }
     }
 
-    protected void cleanupAndGetValidFilesInternal(IODeviceHandle dev, FilenameFilter filter,
+    protected void cleanupAndGetValidFilesInternal(FilenameFilter filter,
             TreeIndexFactory<? extends ITreeIndex> treeFactory, ArrayList<ComparableFileName> allFiles)
             throws HyracksDataException, IndexException {
-        File dir = new File(dev.getPath(), baseDir);
+        File dir = new File(baseDir);
         String[] files = dir.list(filter);
         for (String fileName : files) {
             File file = new File(dir.getPath() + File.separator + fileName);
@@ -110,11 +106,11 @@
         }
     }
 
-    protected void validateFiles(IODeviceHandle dev, HashSet<String> groundTruth,
-            ArrayList<ComparableFileName> validFiles, FilenameFilter filter,
-            TreeIndexFactory<? extends ITreeIndex> treeFactory) throws HyracksDataException, IndexException {
+    protected void validateFiles(HashSet<String> groundTruth, ArrayList<ComparableFileName> validFiles,
+            FilenameFilter filter, TreeIndexFactory<? extends ITreeIndex> treeFactory) throws HyracksDataException,
+            IndexException {
         ArrayList<ComparableFileName> tmpAllInvListsFiles = new ArrayList<ComparableFileName>();
-        cleanupAndGetValidFilesInternal(dev, filter, treeFactory, tmpAllInvListsFiles);
+        cleanupAndGetValidFilesInternal(filter, treeFactory, tmpAllInvListsFiles);
         for (ComparableFileName cmpFileName : tmpAllInvListsFiles) {
             int index = cmpFileName.fileName.lastIndexOf(SPLIT_STRING);
             String file = cmpFileName.fileName.substring(0, index);
@@ -129,13 +125,13 @@
 
     @Override
     public void createDirs() {
-        File f = new File(dev.getPath(), baseDir);
+        File f = new File(baseDir);
         f.mkdirs();
     }
 
     @Override
     public void deleteDirs() {
-        File f = new File(dev.getPath(), baseDir);
+        File f = new File(baseDir);
         delete(f);
     }
 
@@ -155,7 +151,7 @@
     };
 
     protected FileReference createFlushFile(String relFlushFileName) {
-        return dev.createFileReference(relFlushFileName);
+        return new FileReference(new File(relFlushFileName));
     }
 
     protected FileReference createMergeFile(String relMergeFileName) {
@@ -185,12 +181,12 @@
         List<LSMComponentFileReferences> validFiles = new ArrayList<LSMComponentFileReferences>();
         ArrayList<ComparableFileName> allFiles = new ArrayList<ComparableFileName>();
 
-        // Gather files from the IODeviceHandle and delete invalid files
+        // Gather files and delete invalid files
         // There are two types of invalid files:
         // (1) The isValid flag is not set
         // (2) The file's interval is contained by some other file
         // Here, we only filter out (1).
-        cleanupAndGetValidFilesInternal(dev, fileNameFilter, treeFactory, allFiles);
+        cleanupAndGetValidFilesInternal(fileNameFilter, treeFactory, allFiles);
 
         if (allFiles.isEmpty()) {
             return validFiles;
diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexBulkLoadOperatorDescriptor.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexBulkLoadOperatorDescriptor.java
index 425af8d..1c4235c 100644
--- a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexBulkLoadOperatorDescriptor.java
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexBulkLoadOperatorDescriptor.java
@@ -38,13 +38,15 @@
     private final int[] fieldPermutation;
     private final boolean verifyInput;
     private final long numElementsHint;
+    private final boolean checkIfEmptyIndex;
 
     public LSMInvertedIndexBulkLoadOperatorDescriptor(IOperatorDescriptorRegistry spec, int[] fieldPermutation,
-            boolean verifyInput, long numElementsHint, IStorageManagerInterface storageManager,
-            IFileSplitProvider fileSplitProvider, IIndexLifecycleManagerProvider lifecycleManagerProvider,
-            ITypeTraits[] tokenTypeTraits, IBinaryComparatorFactory[] tokenComparatorFactories,
-            ITypeTraits[] invListsTypeTraits, IBinaryComparatorFactory[] invListComparatorFactories,
-            IBinaryTokenizerFactory tokenizerFactory, IIndexDataflowHelperFactory invertedIndexDataflowHelperFactory,
+            boolean verifyInput, long numElementsHint, boolean checkIfEmptyIndex,
+            IStorageManagerInterface storageManager, IFileSplitProvider fileSplitProvider,
+            IIndexLifecycleManagerProvider lifecycleManagerProvider, ITypeTraits[] tokenTypeTraits,
+            IBinaryComparatorFactory[] tokenComparatorFactories, ITypeTraits[] invListsTypeTraits,
+            IBinaryComparatorFactory[] invListComparatorFactories, IBinaryTokenizerFactory tokenizerFactory,
+            IIndexDataflowHelperFactory invertedIndexDataflowHelperFactory,
             IModificationOperationCallbackFactory modificationOpCallbackFactory) {
         super(spec, 1, 0, null, storageManager, fileSplitProvider, lifecycleManagerProvider, tokenTypeTraits,
                 tokenComparatorFactories, invListsTypeTraits, invListComparatorFactories, tokenizerFactory,
@@ -53,12 +55,13 @@
         this.fieldPermutation = fieldPermutation;
         this.verifyInput = verifyInput;
         this.numElementsHint = numElementsHint;
+        this.checkIfEmptyIndex = checkIfEmptyIndex;
     }
 
     @Override
     public IOperatorNodePushable createPushRuntime(IHyracksTaskContext ctx,
             IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) {
         return new IndexBulkLoadOperatorNodePushable(this, ctx, partition, fieldPermutation, 1.0f, verifyInput,
-                numElementsHint, recordDescProvider);
+                numElementsHint, checkIfEmptyIndex, recordDescProvider);
     }
 }
diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexDataflowHelper.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexDataflowHelper.java
index 7ec7c84..1afd9d5 100644
--- a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexDataflowHelper.java
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexDataflowHelper.java
@@ -59,9 +59,8 @@
                     diskFileMapProvider, invIndexOpDesc.getInvListsTypeTraits(),
                     invIndexOpDesc.getInvListsComparatorFactories(), invIndexOpDesc.getTokenTypeTraits(),
                     invIndexOpDesc.getTokenComparatorFactories(), invIndexOpDesc.getTokenizerFactory(),
-                    diskBufferCache, ctx.getIOManager(), file.getFile().getPath(), bloomFilterFalsePositiveRate,
-                    mergePolicy, opTrackerFactory.getOperationTracker(ctx), ioScheduler, ioOpCallbackProvider,
-                    opDesc.getFileSplitProvider().getFileSplits()[partition].getIODeviceId());
+                    diskBufferCache, file.getFile().getPath(), bloomFilterFalsePositiveRate, mergePolicy,
+                    opTrackerFactory.getOperationTracker(ctx), ioScheduler, ioOpCallbackProvider);
             return invIndex;
         } catch (IndexException e) {
             throw new HyracksDataException(e);
diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/dataflow/PartitionedLSMInvertedIndexDataflowHelper.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/dataflow/PartitionedLSMInvertedIndexDataflowHelper.java
index 4a4956f..0dbd06b 100644
--- a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/dataflow/PartitionedLSMInvertedIndexDataflowHelper.java
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/dataflow/PartitionedLSMInvertedIndexDataflowHelper.java
@@ -59,9 +59,8 @@
                     virtualBufferCache, diskFileMapProvider, invIndexOpDesc.getInvListsTypeTraits(),
                     invIndexOpDesc.getInvListsComparatorFactories(), invIndexOpDesc.getTokenTypeTraits(),
                     invIndexOpDesc.getTokenComparatorFactories(), invIndexOpDesc.getTokenizerFactory(),
-                    diskBufferCache, ctx.getIOManager(), file.getFile().getPath(), bloomFilterFalsePositiveRate,
-                    mergePolicy, opTrackerFactory.getOperationTracker(ctx), ioScheduler, ioOpCallbackProvider,
-                    opDesc.getFileSplitProvider().getFileSplits()[partition].getIODeviceId());
+                    diskBufferCache, file.getFile().getPath(), bloomFilterFalsePositiveRate, mergePolicy,
+                    opTrackerFactory.getOperationTracker(ctx), ioScheduler, ioOpCallbackProvider);
             return invIndex;
         } catch (IndexException e) {
             throw new HyracksDataException(e);
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 20d3e7f..3eae5a7 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
@@ -43,7 +43,6 @@
 import edu.uci.ics.hyracks.storage.am.common.api.ISearchPredicate;
 import edu.uci.ics.hyracks.storage.am.common.api.IVirtualFreePageManager;
 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;
@@ -175,6 +174,7 @@
         List<ILSMComponent> immutableComponents = componentsRef.get();
         mutableComponent.getInvIndex().clear();
         mutableComponent.getDeletedKeysBTree().clear();
+        mutableComponent.reset();
         for (ILSMComponent c : immutableComponents) {
             LSMInvertedIndexImmutableComponent component = (LSMInvertedIndexImmutableComponent) c;
             component.getBloomFilter().deactivate();
@@ -432,7 +432,7 @@
         memBTreeAccessor.search(scanCursor, nullPred);
 
         // Bulk load the disk inverted index from the in-memory inverted index.
-        IIndexBulkLoader invIndexBulkLoader = diskInvertedIndex.createBulkLoader(1.0f, false, 0L);
+        IIndexBulkLoader invIndexBulkLoader = diskInvertedIndex.createBulkLoader(1.0f, false, 0L, false);
         try {
             while (scanCursor.hasNext()) {
                 scanCursor.next();
@@ -472,7 +472,7 @@
             deletedKeysBTreeAccessor.search(deletedKeysScanCursor, nullPred);
 
             // Bulk load the deleted-keys BTree.
-            IIndexBulkLoader deletedKeysBTreeBulkLoader = diskDeletedKeysBTree.createBulkLoader(1.0f, false, 0L);
+            IIndexBulkLoader deletedKeysBTreeBulkLoader = diskDeletedKeysBTree.createBulkLoader(1.0f, false, 0L, false);
             IIndexBulkLoader builder = component.getBloomFilter().createBuilder(numBTreeTuples,
                     bloomFilterSpec.getNumHashes(), bloomFilterSpec.getNumBucketsPerElements());
 
@@ -534,7 +534,7 @@
 
         IInvertedIndex mergedDiskInvertedIndex = component.getInvIndex();
         IIndexCursor cursor = mergeOp.getCursor();
-        IIndexBulkLoader invIndexBulkLoader = mergedDiskInvertedIndex.createBulkLoader(1.0f, true, 0L);
+        IIndexBulkLoader invIndexBulkLoader = mergedDiskInvertedIndex.createBulkLoader(1.0f, true, 0L, false);
         try {
             while (cursor.hasNext()) {
                 cursor.next();
@@ -559,63 +559,76 @@
     }
 
     @Override
-    public IIndexBulkLoader createBulkLoader(float fillFactor, boolean verifyInput, long numElementsHint)
-            throws IndexException {
-        return new LSMInvertedIndexBulkLoader(fillFactor, verifyInput, numElementsHint);
+    public IIndexBulkLoader createBulkLoader(float fillFactor, boolean verifyInput, long numElementsHint,
+            boolean checkIfEmptyIndex) throws IndexException {
+        try {
+            return new LSMInvertedIndexBulkLoader(fillFactor, verifyInput, numElementsHint, checkIfEmptyIndex);
+        } catch (HyracksDataException e) {
+            throw new IndexException(e);
+        }
+    }
+
+    public boolean isEmptyIndex() throws HyracksDataException {
+        return componentsRef.get().isEmpty() && !mutableComponent.isModified();
     }
 
     public class LSMInvertedIndexBulkLoader implements IIndexBulkLoader {
         private final ILSMComponent component;
         private final IIndexBulkLoader invIndexBulkLoader;
-        private boolean exceptionCaught = false;
+        private boolean cleanedUpArtifacts = false;
+        private boolean isEmptyComponent = true;
 
-        public LSMInvertedIndexBulkLoader(float fillFactor, boolean verifyInput, long numElementsHint)
-                throws IndexException {
+        public LSMInvertedIndexBulkLoader(float fillFactor, boolean verifyInput, long numElementsHint,
+                boolean checkIfEmptyIndex) throws IndexException, HyracksDataException {
+            if (checkIfEmptyIndex && !isEmptyIndex()) {
+                throw new IndexException("Cannot load an index that is not empty");
+            }
             // Note that by using a flush target file name, we state that the
             // new bulk loaded tree is "newer" than any other merged tree.
             try {
                 component = createBulkLoadTarget();
-            } catch (HyracksDataException e) {
-                throw new TreeIndexException(e);
-            } catch (IndexException e) {
-                throw new TreeIndexException(e);
+            } catch (HyracksDataException | IndexException e) {
+                throw new IndexException(e);
             }
             invIndexBulkLoader = ((LSMInvertedIndexImmutableComponent) component).getInvIndex().createBulkLoader(
-                    fillFactor, verifyInput, numElementsHint);
+                    fillFactor, verifyInput, numElementsHint, false);
         }
 
         @Override
         public void add(ITupleReference tuple) throws IndexException, HyracksDataException {
             try {
                 invIndexBulkLoader.add(tuple);
-            } catch (IndexException e) {
-                handleException();
+            } catch (IndexException | HyracksDataException | RuntimeException e) {
+                cleanupArtifacts();
                 throw e;
-            } catch (HyracksDataException e) {
-                handleException();
-                throw e;
-            } catch (RuntimeException e) {
-                handleException();
-                throw e;
+            }
+            if (isEmptyComponent) {
+                isEmptyComponent = false;
             }
         }
 
-        protected void handleException() throws HyracksDataException {
-            exceptionCaught = true;
-            ((LSMInvertedIndexImmutableComponent) component).getInvIndex().deactivate();
-            ((LSMInvertedIndexImmutableComponent) component).getInvIndex().destroy();
-            ((LSMInvertedIndexImmutableComponent) component).getDeletedKeysBTree().deactivate();
-            ((LSMInvertedIndexImmutableComponent) component).getDeletedKeysBTree().destroy();
-            ((LSMInvertedIndexImmutableComponent) component).getBloomFilter().deactivate();
-            ((LSMInvertedIndexImmutableComponent) component).getBloomFilter().destroy();
+        protected void cleanupArtifacts() throws HyracksDataException {
+            if (!cleanedUpArtifacts) {
+                cleanedUpArtifacts = true;
+                ((LSMInvertedIndexImmutableComponent) component).getInvIndex().deactivate();
+                ((LSMInvertedIndexImmutableComponent) component).getInvIndex().destroy();
+                ((LSMInvertedIndexImmutableComponent) component).getDeletedKeysBTree().deactivate();
+                ((LSMInvertedIndexImmutableComponent) component).getDeletedKeysBTree().destroy();
+                ((LSMInvertedIndexImmutableComponent) component).getBloomFilter().deactivate();
+                ((LSMInvertedIndexImmutableComponent) component).getBloomFilter().destroy();
+            }
         }
 
         @Override
         public void end() throws IndexException, HyracksDataException {
-            if (!exceptionCaught) {
+            if (!cleanedUpArtifacts) {
                 invIndexBulkLoader.end();
+                if (isEmptyComponent) {
+                    cleanupArtifacts();
+                } else {
+                    lsmHarness.addBulkLoadedComponent(component);
+                }
             }
-            lsmHarness.addBulkLoadedComponent(component);
         }
     }
 
diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexFileManager.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexFileManager.java
index db6c337..825e5d3 100644
--- a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexFileManager.java
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexFileManager.java
@@ -26,7 +26,6 @@
 
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.api.io.FileReference;
-import edu.uci.ics.hyracks.api.io.IIOManager;
 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.BTreeFactory;
@@ -61,9 +60,8 @@
         }
     };
 
-    public LSMInvertedIndexFileManager(IIOManager ioManager, IFileMapProvider fileMapProvider, FileReference file,
-            BTreeFactory btreeFactory, int ioDeviceId) {
-        super(ioManager, fileMapProvider, file, null, ioDeviceId);
+    public LSMInvertedIndexFileManager(IFileMapProvider fileMapProvider, FileReference file, BTreeFactory btreeFactory) {
+        super(fileMapProvider, file, null);
         this.btreeFactory = btreeFactory;
     }
 
@@ -99,8 +97,8 @@
         ArrayList<ComparableFileName> allDeletedKeysBTreeFiles = new ArrayList<ComparableFileName>();
         ArrayList<ComparableFileName> allBloomFilterFiles = new ArrayList<ComparableFileName>();
 
-        // Gather files from the IODeviceHandle.
-        cleanupAndGetValidFilesInternal(dev, deletedKeysBTreeFilter, btreeFactory, allDeletedKeysBTreeFiles);
+        // Gather files.
+        cleanupAndGetValidFilesInternal(deletedKeysBTreeFilter, btreeFactory, allDeletedKeysBTreeFiles);
         HashSet<String> deletedKeysBTreeFilesSet = new HashSet<String>();
         for (ComparableFileName cmpFileName : allDeletedKeysBTreeFiles) {
             int index = cmpFileName.fileName.lastIndexOf(SPLIT_STRING);
@@ -108,9 +106,9 @@
         }
 
         // TODO: do we really need to validate the inverted lists files or is validating the dict. BTrees is enough?
-        validateFiles(dev, deletedKeysBTreeFilesSet, allInvListsFiles, invListFilter, null);
-        validateFiles(dev, deletedKeysBTreeFilesSet, allDictBTreeFiles, dictBTreeFilter, btreeFactory);
-        validateFiles(dev, deletedKeysBTreeFilesSet, allBloomFilterFiles, bloomFilterFilter, null);
+        validateFiles(deletedKeysBTreeFilesSet, allInvListsFiles, invListFilter, null);
+        validateFiles(deletedKeysBTreeFilesSet, allDictBTreeFiles, dictBTreeFilter, btreeFactory);
+        validateFiles(deletedKeysBTreeFilesSet, allBloomFilterFiles, bloomFilterFilter, null);
 
         // Sanity check.
         if (allDictBTreeFiles.size() != allInvListsFiles.size()
@@ -151,30 +149,30 @@
         validComparableBloomFilterFiles.add(lastBloomFilter);
 
         for (int i = 1; i < allDictBTreeFiles.size(); i++) {
-            ComparableFileName currentRTree = allDictBTreeFiles.get(i);
-            ComparableFileName currentBTree = allDictBTreeFiles.get(i);
+            ComparableFileName currentDeletedKeysBTree = allDeletedKeysBTreeFiles.get(i);
+            ComparableFileName CurrentDictBTree = allDictBTreeFiles.get(i);
             ComparableFileName currentBloomFilter = allBloomFilterFiles.get(i);
             // Current start timestamp is greater than last stop timestamp.
-            if (currentRTree.interval[0].compareTo(lastDeletedKeysBTree.interval[1]) > 0
-                    && currentBTree.interval[0].compareTo(lastDeletedKeysBTree.interval[1]) > 0
+            if (currentDeletedKeysBTree.interval[0].compareTo(lastDeletedKeysBTree.interval[1]) > 0
+                    && CurrentDictBTree.interval[0].compareTo(lastDictBTree.interval[1]) > 0
                     && currentBloomFilter.interval[0].compareTo(lastBloomFilter.interval[1]) > 0) {
-                validComparableDictBTreeFiles.add(currentRTree);
-                validComparableDeletedKeysBTreeFiles.add(currentBTree);
+                validComparableDictBTreeFiles.add(CurrentDictBTree);
+                validComparableDeletedKeysBTreeFiles.add(currentDeletedKeysBTree);
                 validComparableBloomFilterFiles.add(currentBloomFilter);
-                lastDictBTree = currentRTree;
-                lastDeletedKeysBTree = currentBTree;
+                lastDictBTree = CurrentDictBTree;
+                lastDeletedKeysBTree = currentDeletedKeysBTree;
                 lastBloomFilter = currentBloomFilter;
-            } else if (currentRTree.interval[0].compareTo(lastDictBTree.interval[0]) >= 0
-                    && currentRTree.interval[1].compareTo(lastDictBTree.interval[1]) <= 0
-                    && currentBTree.interval[0].compareTo(lastDeletedKeysBTree.interval[0]) >= 0
-                    && currentBTree.interval[1].compareTo(lastDeletedKeysBTree.interval[1]) <= 0
+            } else if (currentDeletedKeysBTree.interval[0].compareTo(lastDeletedKeysBTree.interval[0]) >= 0
+                    && currentDeletedKeysBTree.interval[1].compareTo(lastDeletedKeysBTree.interval[1]) <= 0
+                    && CurrentDictBTree.interval[0].compareTo(lastDictBTree.interval[0]) >= 0
+                    && CurrentDictBTree.interval[1].compareTo(lastDictBTree.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 invalidRTreeFile = new File(currentRTree.fullPath);
-                invalidRTreeFile.delete();
-                File invalidBTreeFile = new File(currentBTree.fullPath);
-                invalidBTreeFile.delete();
+                File invalidDeletedBTreeFile = new File(currentDeletedKeysBTree.fullPath);
+                invalidDeletedBTreeFile.delete();
+                File invalidDictBTreeFile = new File(CurrentDictBTree.fullPath);
+                invalidDictBTreeFile.delete();
                 File invalidBloomFilterFile = new File(currentBloomFilter.fullPath);
                 invalidBloomFilterFile.delete();
             } else {
diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexOpContext.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexOpContext.java
index 9e5f75b..fd83bfb 100644
--- a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexOpContext.java
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexOpContext.java
@@ -18,6 +18,7 @@
 import java.util.LinkedList;
 import java.util.List;
 
+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.IIndexAccessor;
 import edu.uci.ics.hyracks.storage.am.common.api.IModificationOperationCallback;
@@ -66,7 +67,7 @@
 
     @Override
     // TODO: Ignore opcallback for now.
-    public void setOperation(IndexOperation newOp) {
+    public void setOperation(IndexOperation newOp) throws HyracksDataException {
         reset();
         switch (newOp) {
             case INSERT:
diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexSearchCursor.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexSearchCursor.java
index b9f50a1..882d1a1 100644
--- a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexSearchCursor.java
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexSearchCursor.java
@@ -165,14 +165,7 @@
 
     @Override
     public void close() throws HyracksDataException {
-        try {
-            reset();
-            accessorIndex = -1;
-        } finally {
-            if (harness != null) {
-                harness.endSearch(opCtx);
-            }
-        }
+        reset();
     }
 
     @Override
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 068031d..222f4de 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
@@ -163,7 +163,7 @@
 
     @Override
     public IIndexAccessor createAccessor(IModificationOperationCallback modificationCallback,
-            ISearchOperationCallback searchCallback) {
+            ISearchOperationCallback searchCallback) throws HyracksDataException {
         return new InMemoryInvertedIndexAccessor(this, new InMemoryInvertedIndexOpContext(btree, tokenCmpFactories,
                 tokenizerFactory));
     }
@@ -188,8 +188,8 @@
     }
 
     @Override
-    public IIndexBulkLoader createBulkLoader(float fillFactor, boolean verifyInput, long numElementsHint)
-            throws IndexException {
+    public IIndexBulkLoader createBulkLoader(float fillFactor, boolean verifyInput, long numElementsHint,
+            boolean checkIfEmptyIndex) throws IndexException {
         throw new UnsupportedOperationException("Bulk load not supported by in-memory inverted index.");
     }
 
diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/inmemory/InMemoryInvertedIndexAccessor.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/inmemory/InMemoryInvertedIndexAccessor.java
index fe8dfd8..7e9b483 100644
--- a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/inmemory/InMemoryInvertedIndexAccessor.java
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/inmemory/InMemoryInvertedIndexAccessor.java
@@ -43,7 +43,8 @@
     protected InMemoryInvertedIndex index;
     protected BTreeAccessor btreeAccessor;
 
-    public InMemoryInvertedIndexAccessor(InMemoryInvertedIndex index, IIndexOperationContext opCtx) {
+    public InMemoryInvertedIndexAccessor(InMemoryInvertedIndex index, IIndexOperationContext opCtx)
+            throws HyracksDataException {
         this.opCtx = opCtx;
         this.index = index;
         this.searcher = createSearcher();
@@ -110,7 +111,7 @@
         throw new UnsupportedOperationException("Upsert not supported by in-memory inverted index.");
     }
 
-    protected IInvertedIndexSearcher createSearcher() {
+    protected IInvertedIndexSearcher createSearcher() throws HyracksDataException {
         return new TOccurrenceSearcher(hyracksCtx, index);
     }
 }
diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/inmemory/PartitionedInMemoryInvertedIndex.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/inmemory/PartitionedInMemoryInvertedIndex.java
index ae86990..eb06758 100644
--- a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/inmemory/PartitionedInMemoryInvertedIndex.java
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/inmemory/PartitionedInMemoryInvertedIndex.java
@@ -83,7 +83,7 @@
 
     @Override
     public IIndexAccessor createAccessor(IModificationOperationCallback modificationCallback,
-            ISearchOperationCallback searchCallback) {
+            ISearchOperationCallback searchCallback) throws HyracksDataException {
         return new PartitionedInMemoryInvertedIndexAccessor(this, new PartitionedInMemoryInvertedIndexOpContext(btree,
                 tokenCmpFactories, tokenizerFactory));
     }
diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/inmemory/PartitionedInMemoryInvertedIndexAccessor.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/inmemory/PartitionedInMemoryInvertedIndexAccessor.java
index 133df88..6b8bf00 100644
--- a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/inmemory/PartitionedInMemoryInvertedIndexAccessor.java
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/inmemory/PartitionedInMemoryInvertedIndexAccessor.java
@@ -15,17 +15,19 @@
 
 package edu.uci.ics.hyracks.storage.am.lsm.invertedindex.inmemory;
 
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.storage.am.common.api.IIndexOperationContext;
 import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.api.IInvertedIndexSearcher;
 import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.search.PartitionedTOccurrenceSearcher;
 
 public class PartitionedInMemoryInvertedIndexAccessor extends InMemoryInvertedIndexAccessor {
 
-    public PartitionedInMemoryInvertedIndexAccessor(InMemoryInvertedIndex index, IIndexOperationContext opCtx) {
+    public PartitionedInMemoryInvertedIndexAccessor(InMemoryInvertedIndex index, IIndexOperationContext opCtx)
+            throws HyracksDataException {
         super(index, opCtx);
     }
 
-    protected IInvertedIndexSearcher createSearcher() {
+    protected IInvertedIndexSearcher createSearcher() throws HyracksDataException {
         return new PartitionedTOccurrenceSearcher(hyracksCtx, index);
     }
 }
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 5f75f8d..0ab2f4d 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
@@ -303,7 +303,7 @@
         private final MultiComparator allCmp;
 
         public OnDiskInvertedIndexBulkLoader(float btreeFillFactor, boolean verifyInput, long numElementsHint,
-                int startPageId, int fileId) throws IndexException, HyracksDataException {
+                boolean checkIfEmptyIndex, int startPageId, int fileId) throws IndexException, HyracksDataException {
             this.verifyInput = verifyInput;
             this.tokenCmp = MultiComparator.create(btree.getComparatorFactories());
             this.invListCmp = MultiComparator.create(invListCmpFactories);
@@ -316,7 +316,8 @@
             this.btreeTupleReference = new ArrayTupleReference();
             this.lastTupleBuilder = new ArrayTupleBuilder(numTokenFields + numInvListKeys);
             this.lastTuple = new ArrayTupleReference();
-            this.btreeBulkloader = btree.createBulkLoader(btreeFillFactor, verifyInput, numElementsHint);
+            this.btreeBulkloader = btree.createBulkLoader(btreeFillFactor, verifyInput, numElementsHint,
+                    checkIfEmptyIndex);
             currentPageId = startPageId;
             currentPage = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, currentPageId), true);
             currentPage.acquireWriteLatch();
@@ -467,7 +468,7 @@
         private final IInvertedIndexSearcher searcher;
         private final IIndexOperationContext opCtx = new OnDiskInvertedIndexOpContext(btree);
 
-        public OnDiskInvertedIndexAccessor(OnDiskInvertedIndex index) {
+        public OnDiskInvertedIndexAccessor(OnDiskInvertedIndex index) throws HyracksDataException {
             this.index = index;
             this.searcher = new TOccurrenceSearcher(ctx, index);
         }
@@ -535,7 +536,7 @@
 
     @Override
     public IIndexAccessor createAccessor(IModificationOperationCallback modificationCallback,
-            ISearchOperationCallback searchCallback) {
+            ISearchOperationCallback searchCallback) throws HyracksDataException {
         return new OnDiskInvertedIndexAccessor(this);
     }
 
@@ -563,10 +564,11 @@
     }
 
     @Override
-    public IIndexBulkLoader createBulkLoader(float fillFactor, boolean verifyInput, long numElementsHint)
-            throws IndexException {
+    public IIndexBulkLoader createBulkLoader(float fillFactor, boolean verifyInput, long numElementsHint,
+            boolean checkIfEmptyIndex) throws IndexException {
         try {
-            return new OnDiskInvertedIndexBulkLoader(fillFactor, verifyInput, numElementsHint, rootPageId, fileId);
+            return new OnDiskInvertedIndexBulkLoader(fillFactor, verifyInput, numElementsHint, checkIfEmptyIndex,
+                    rootPageId, fileId);
         } catch (HyracksDataException e) {
             throw new InvertedIndexException(e);
         }
diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/ondisk/PartitionedOnDiskInvertedIndex.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/ondisk/PartitionedOnDiskInvertedIndex.java
index 2b2a174..6f7873b 100644
--- a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/ondisk/PartitionedOnDiskInvertedIndex.java
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/ondisk/PartitionedOnDiskInvertedIndex.java
@@ -51,14 +51,14 @@
     }
 
     public class PartitionedOnDiskInvertedIndexAccessor extends OnDiskInvertedIndexAccessor {
-        public PartitionedOnDiskInvertedIndexAccessor(OnDiskInvertedIndex index) {
+        public PartitionedOnDiskInvertedIndexAccessor(OnDiskInvertedIndex index) throws HyracksDataException {
             super(index, new PartitionedTOccurrenceSearcher(ctx, index));
         }
     }
 
     @Override
     public IIndexAccessor createAccessor(IModificationOperationCallback modificationCallback,
-            ISearchOperationCallback searchCallback) {
+            ISearchOperationCallback searchCallback) throws HyracksDataException {
         return new PartitionedOnDiskInvertedIndexAccessor(this);
     }
 
diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/search/AbstractTOccurrenceSearcher.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/search/AbstractTOccurrenceSearcher.java
index 44d2b5d..2bbf1f3 100644
--- a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/search/AbstractTOccurrenceSearcher.java
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/search/AbstractTOccurrenceSearcher.java
@@ -68,7 +68,7 @@
     protected final IObjectFactory<IInvertedListCursor> invListCursorFactory;
     protected final ObjectCache<IInvertedListCursor> invListCursorCache;
 
-    public AbstractTOccurrenceSearcher(IHyracksCommonContext ctx, IInvertedIndex invIndex) {
+    public AbstractTOccurrenceSearcher(IHyracksCommonContext ctx, IInvertedIndex invIndex) throws HyracksDataException {
         this.ctx = ctx;
         this.invListMerger = new InvertedListMerger(ctx, invIndex);
         this.searchResult = new SearchResult(invIndex.getInvListTypeTraits(), ctx);
diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/search/InvertedListMerger.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/search/InvertedListMerger.java
index a6b825a..692a640 100644
--- a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/search/InvertedListMerger.java
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/search/InvertedListMerger.java
@@ -38,7 +38,7 @@
     protected SearchResult prevSearchResult;
     protected SearchResult newSearchResult;
 
-    public InvertedListMerger(IHyracksCommonContext ctx, IInvertedIndex invIndex) {
+    public InvertedListMerger(IHyracksCommonContext ctx, IInvertedIndex invIndex) throws HyracksDataException {
         this.invListCmp = MultiComparator.createIgnoreFieldLength(invIndex.getInvListCmpFactories());
         this.prevSearchResult = new SearchResult(invIndex.getInvListTypeTraits(), ctx);
         this.newSearchResult = new SearchResult(prevSearchResult);
@@ -72,11 +72,9 @@
                 int currentNumResults = prevSearchResult.getNumResults();
                 // Should we binary search the next list or should we sort-merge it?
                 if (currentNumResults * Math.log(numInvListElements) < currentNumResults + numInvListElements) {
-                    mergeSuffixListProbe(invListCursor, prevSearchResult, result, i, numInvLists,
-                            occurrenceThreshold);
+                    mergeSuffixListProbe(invListCursor, prevSearchResult, result, i, numInvLists, occurrenceThreshold);
                 } else {
-                    mergeSuffixListScan(invListCursor, prevSearchResult, result, i, numInvLists,
-                            occurrenceThreshold);
+                    mergeSuffixListScan(invListCursor, prevSearchResult, result, i, numInvLists, occurrenceThreshold);
                 }
             }
             invListCursor.unpinPages();
@@ -319,7 +317,7 @@
         }
     }
 
-    public SearchResult createSearchResult() {
+    public SearchResult createSearchResult() throws HyracksDataException {
         return new SearchResult(prevSearchResult);
     }
 
diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/search/PartitionedTOccurrenceSearcher.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/search/PartitionedTOccurrenceSearcher.java
index 18313c1..adad532 100644
--- a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/search/PartitionedTOccurrenceSearcher.java
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/search/PartitionedTOccurrenceSearcher.java
@@ -49,7 +49,8 @@
     protected final ArrayList<IInvertedListCursor> cursorsOrderedByTokens = new ArrayList<IInvertedListCursor>();
     protected final InvertedListPartitions partitions = new InvertedListPartitions();
 
-    public PartitionedTOccurrenceSearcher(IHyracksCommonContext ctx, IInvertedIndex invIndex) {
+    public PartitionedTOccurrenceSearcher(IHyracksCommonContext ctx, IInvertedIndex invIndex)
+            throws HyracksDataException {
         super(ctx, invIndex);
         initHelperTuples();
     }
@@ -89,19 +90,19 @@
         if (partInvIndex.isEmpty()) {
             return;
         }
-        
+
         tokenizeQuery(searchPred);
         short numQueryTokens = (short) queryTokenAccessor.getTupleCount();
 
         IInvertedIndexSearchModifier searchModifier = searchPred.getSearchModifier();
         short numTokensLowerBound = searchModifier.getNumTokensLowerBound(numQueryTokens);
         short numTokensUpperBound = searchModifier.getNumTokensUpperBound(numQueryTokens);
-        
+
         occurrenceThreshold = searchModifier.getOccurrenceThreshold(numQueryTokens);
         if (occurrenceThreshold <= 0) {
             throw new OccurrenceThresholdPanicException("Merge Threshold is <= 0. Failing Search.");
         }
-        
+
         short maxCountPossible = numQueryTokens;
         invListCursorCache.reset();
         partitions.reset(numTokensLowerBound, numTokensUpperBound);
@@ -112,16 +113,16 @@
                     partitions, cursorsOrderedByTokens)) {
                 maxCountPossible--;
                 // No results possible.
-                if (maxCountPossible < occurrenceThreshold) {                    
+                if (maxCountPossible < occurrenceThreshold) {
                     return;
                 }
             }
         }
-        
+
         ArrayList<IInvertedListCursor>[] partitionCursors = partitions.getPartitions();
         short start = partitions.getMinValidPartitionIndex();
         short end = partitions.getMaxValidPartitionIndex();
-        
+
         // Typically, we only enter this case for disk-based inverted indexes. 
         // TODO: This behavior could potentially lead to a deadlock if we cannot pin 
         // all inverted lists in memory, and are forced to wait for a page to get evicted
@@ -145,7 +146,7 @@
                 cursorsOrderedByTokens.get(i).pinPages();
             }
         }
-        
+
         // Process the partitions one-by-one.
         for (int i = start; i <= end; i++) {
             if (partitionCursors[i] == null) {
@@ -160,7 +161,7 @@
             invListMerger.reset();
             invListMerger.merge(partitionCursors[i], occurrenceThreshold, numPrefixLists, searchResult);
         }
-        
+
         resultCursor.open(null, searchPred);
     }
 
diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/search/SearchResult.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/search/SearchResult.java
index b091195..c4056c5 100644
--- a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/search/SearchResult.java
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/search/SearchResult.java
@@ -20,6 +20,7 @@
 
 import edu.uci.ics.hyracks.api.context.IHyracksCommonContext;
 import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.data.std.primitive.IntegerPointable;
 import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
 import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.ondisk.FixedSizeFrameTupleAccessor;
@@ -42,7 +43,7 @@
     protected int currBufIdx;
     protected int numResults;
 
-    public SearchResult(ITypeTraits[] invListFields, IHyracksCommonContext ctx) {
+    public SearchResult(ITypeTraits[] invListFields, IHyracksCommonContext ctx) throws HyracksDataException {
         typeTraits = new ITypeTraits[invListFields.length + 1];
         int tmp = 0;
         for (int i = 0; i < invListFields.length; i++) {
@@ -61,8 +62,10 @@
 
     /**
      * Initialize from other search-result object to share member instances except for result buffers.
+     * 
+     * @throws HyracksDataException
      */
-    public SearchResult(SearchResult other) {
+    public SearchResult(SearchResult other) throws HyracksDataException {
         this.ctx = other.ctx;
         this.appender = other.appender;
         this.accessor = other.accessor;
@@ -102,7 +105,7 @@
         }
     }
 
-    public void append(ITupleReference invListElement, int count) {
+    public void append(ITupleReference invListElement, int count) throws HyracksDataException {
         ByteBuffer currentBuffer = buffers.get(currBufIdx);
         if (!appender.hasSpace()) {
             currBufIdx++;
diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/search/TOccurrenceSearcher.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/search/TOccurrenceSearcher.java
index 25552ea..348ef75 100644
--- a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/search/TOccurrenceSearcher.java
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/search/TOccurrenceSearcher.java
@@ -31,7 +31,7 @@
 
     protected final ArrayList<IInvertedListCursor> invListCursors = new ArrayList<IInvertedListCursor>();
 
-    public TOccurrenceSearcher(IHyracksCommonContext ctx, IInvertedIndex invIndex) {
+    public TOccurrenceSearcher(IHyracksCommonContext ctx, IInvertedIndex invIndex) throws HyracksDataException {
         super(ctx, invIndex);
     }
 
diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/util/InvertedIndexUtils.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/util/InvertedIndexUtils.java
index 17a6742..9d85f56 100644
--- a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/util/InvertedIndexUtils.java
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/util/InvertedIndexUtils.java
@@ -20,7 +20,6 @@
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
 import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
 import edu.uci.ics.hyracks.api.io.FileReference;
-import edu.uci.ics.hyracks.api.io.IIOManager;
 import edu.uci.ics.hyracks.storage.am.bloomfilter.impls.BloomFilterFactory;
 import edu.uci.ics.hyracks.storage.am.btree.exceptions.BTreeException;
 import edu.uci.ics.hyracks.storage.am.btree.frames.BTreeLeafFrameType;
@@ -120,21 +119,9 @@
             IFileMapProvider diskFileMapProvider, ITypeTraits[] invListTypeTraits,
             IBinaryComparatorFactory[] invListCmpFactories, ITypeTraits[] tokenTypeTraits,
             IBinaryComparatorFactory[] tokenCmpFactories, IBinaryTokenizerFactory tokenizerFactory,
-            IBufferCache diskBufferCache, IIOManager ioManager, String onDiskDir, double bloomFilterFalsePositiveRate,
+            IBufferCache diskBufferCache, String onDiskDir, double bloomFilterFalsePositiveRate,
             ILSMMergePolicy mergePolicy, ILSMOperationTracker opTracker, ILSMIOOperationScheduler ioScheduler,
             ILSMIOOperationCallbackProvider ioOpCallbackProvider) throws IndexException {
-        return createLSMInvertedIndex(virtualBufferCache, diskFileMapProvider, invListTypeTraits, invListCmpFactories,
-                tokenTypeTraits, tokenCmpFactories, tokenizerFactory, diskBufferCache, ioManager, onDiskDir,
-                bloomFilterFalsePositiveRate, mergePolicy, opTracker, ioScheduler, ioOpCallbackProvider, 0);
-    }
-
-    public static LSMInvertedIndex createLSMInvertedIndex(IVirtualBufferCache virtualBufferCache,
-            IFileMapProvider diskFileMapProvider, ITypeTraits[] invListTypeTraits,
-            IBinaryComparatorFactory[] invListCmpFactories, ITypeTraits[] tokenTypeTraits,
-            IBinaryComparatorFactory[] tokenCmpFactories, IBinaryTokenizerFactory tokenizerFactory,
-            IBufferCache diskBufferCache, IIOManager ioManager, String onDiskDir, double bloomFilterFalsePositiveRate,
-            ILSMMergePolicy mergePolicy, ILSMOperationTracker opTracker, ILSMIOOperationScheduler ioScheduler,
-            ILSMIOOperationCallbackProvider ioOpCallbackProvider, int ioDeviceId) throws IndexException {
 
         BTreeFactory deletedKeysBTreeFactory = createDeletedKeysBTreeFactory(diskFileMapProvider, invListTypeTraits,
                 invListCmpFactories, diskBufferCache);
@@ -147,8 +134,8 @@
                 bloomFilterKeyFields);
 
         FileReference onDiskDirFileRef = new FileReference(new File(onDiskDir));
-        LSMInvertedIndexFileManager fileManager = new LSMInvertedIndexFileManager(ioManager, diskFileMapProvider,
-                onDiskDirFileRef, deletedKeysBTreeFactory, ioDeviceId);
+        LSMInvertedIndexFileManager fileManager = new LSMInvertedIndexFileManager(diskFileMapProvider,
+                onDiskDirFileRef, deletedKeysBTreeFactory);
 
         IInvertedListBuilderFactory invListBuilderFactory = new FixedSizeElementInvertedListBuilderFactory(
                 invListTypeTraits);
@@ -167,21 +154,9 @@
             IFileMapProvider diskFileMapProvider, ITypeTraits[] invListTypeTraits,
             IBinaryComparatorFactory[] invListCmpFactories, ITypeTraits[] tokenTypeTraits,
             IBinaryComparatorFactory[] tokenCmpFactories, IBinaryTokenizerFactory tokenizerFactory,
-            IBufferCache diskBufferCache, IIOManager ioManager, String onDiskDir, double bloomFilterFalsePositiveRate,
+            IBufferCache diskBufferCache, String onDiskDir, double bloomFilterFalsePositiveRate,
             ILSMMergePolicy mergePolicy, ILSMOperationTracker opTracker, ILSMIOOperationScheduler ioScheduler,
             ILSMIOOperationCallbackProvider ioOpCallbackProvider) throws IndexException {
-        return createPartitionedLSMInvertedIndex(virtualBufferCache, diskFileMapProvider, invListTypeTraits,
-                invListCmpFactories, tokenTypeTraits, tokenCmpFactories, tokenizerFactory, diskBufferCache, ioManager,
-                onDiskDir, bloomFilterFalsePositiveRate, mergePolicy, opTracker, ioScheduler, ioOpCallbackProvider, 0);
-    }
-
-    public static PartitionedLSMInvertedIndex createPartitionedLSMInvertedIndex(IVirtualBufferCache virtualBufferCache,
-            IFileMapProvider diskFileMapProvider, ITypeTraits[] invListTypeTraits,
-            IBinaryComparatorFactory[] invListCmpFactories, ITypeTraits[] tokenTypeTraits,
-            IBinaryComparatorFactory[] tokenCmpFactories, IBinaryTokenizerFactory tokenizerFactory,
-            IBufferCache diskBufferCache, IIOManager ioManager, String onDiskDir, double bloomFilterFalsePositiveRate,
-            ILSMMergePolicy mergePolicy, ILSMOperationTracker opTracker, ILSMIOOperationScheduler ioScheduler,
-            ILSMIOOperationCallbackProvider ioOpCallbackProvider, int ioDeviceId) throws IndexException {
 
         BTreeFactory deletedKeysBTreeFactory = createDeletedKeysBTreeFactory(diskFileMapProvider, invListTypeTraits,
                 invListCmpFactories, diskBufferCache);
@@ -194,8 +169,8 @@
                 bloomFilterKeyFields);
 
         FileReference onDiskDirFileRef = new FileReference(new File(onDiskDir));
-        LSMInvertedIndexFileManager fileManager = new LSMInvertedIndexFileManager(ioManager, diskFileMapProvider,
-                onDiskDirFileRef, deletedKeysBTreeFactory, ioDeviceId);
+        LSMInvertedIndexFileManager fileManager = new LSMInvertedIndexFileManager(diskFileMapProvider,
+                onDiskDirFileRef, deletedKeysBTreeFactory);
 
         IInvertedListBuilderFactory invListBuilderFactory = new FixedSizeElementInvertedListBuilderFactory(
                 invListTypeTraits);
diff --git a/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/dataflow/AbstractLSMRTreeDataflowHelper.java b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/dataflow/AbstractLSMRTreeDataflowHelper.java
index 169efc4..9fc8cea 100644
--- a/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/dataflow/AbstractLSMRTreeDataflowHelper.java
+++ b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/dataflow/AbstractLSMRTreeDataflowHelper.java
@@ -21,7 +21,6 @@
 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.api.io.IIOManager;
 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.dataflow.AbstractTreeIndexOperatorDescriptor;
@@ -73,18 +72,17 @@
     @Override
     public ITreeIndex createIndexInstance() throws HyracksDataException {
         AbstractTreeIndexOperatorDescriptor treeOpDesc = (AbstractTreeIndexOperatorDescriptor) opDesc;
-        return createLSMTree(virtualBufferCache, ctx.getIOManager(), file,
-                opDesc.getStorageManager().getBufferCache(ctx), opDesc.getStorageManager().getFileMapProvider(ctx),
-                treeOpDesc.getTreeIndexTypeTraits(), treeOpDesc.getTreeIndexComparatorFactories(),
-                btreeComparatorFactories, opTrackerFactory.getOperationTracker(ctx), valueProviderFactories,
-                rtreePolicyType, linearizeCmpFactory, partition);
+        return createLSMTree(virtualBufferCache, file, opDesc.getStorageManager().getBufferCache(ctx), opDesc
+                .getStorageManager().getFileMapProvider(ctx), treeOpDesc.getTreeIndexTypeTraits(),
+                treeOpDesc.getTreeIndexComparatorFactories(), btreeComparatorFactories,
+                opTrackerFactory.getOperationTracker(ctx), valueProviderFactories, rtreePolicyType, linearizeCmpFactory);
 
     }
 
-    protected abstract ITreeIndex createLSMTree(IVirtualBufferCache virtualBufferCache, IIOManager ioManager,
-            FileReference file, IBufferCache diskBufferCache, IFileMapProvider diskFileMapProvider,
-            ITypeTraits[] typeTraits, IBinaryComparatorFactory[] rtreeCmpFactories,
-            IBinaryComparatorFactory[] btreeCmpFactories, ILSMOperationTracker opTracker,
-            IPrimitiveValueProviderFactory[] valueProviderFactories, RTreePolicyType rtreePolicyType,
-            ILinearizeComparatorFactory linearizeCmpFactory, int startIODeviceIndex) throws HyracksDataException;
+    protected abstract ITreeIndex createLSMTree(IVirtualBufferCache virtualBufferCache, FileReference file,
+            IBufferCache diskBufferCache, IFileMapProvider diskFileMapProvider, ITypeTraits[] typeTraits,
+            IBinaryComparatorFactory[] rtreeCmpFactories, IBinaryComparatorFactory[] btreeCmpFactories,
+            ILSMOperationTracker opTracker, IPrimitiveValueProviderFactory[] valueProviderFactories,
+            RTreePolicyType rtreePolicyType, ILinearizeComparatorFactory linearizeCmpFactory)
+            throws HyracksDataException;
 }
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 f8b6792..b47bc02 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
@@ -21,7 +21,6 @@
 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.api.io.IIOManager;
 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;
@@ -62,18 +61,17 @@
     }
 
     @Override
-    protected ITreeIndex createLSMTree(IVirtualBufferCache virtualBufferCache, IIOManager ioManager,
-            FileReference file, IBufferCache diskBufferCache, IFileMapProvider diskFileMapProvider,
-            ITypeTraits[] typeTraits, IBinaryComparatorFactory[] rtreeCmpFactories,
-            IBinaryComparatorFactory[] btreeCmpFactories, ILSMOperationTracker opTracker,
-            IPrimitiveValueProviderFactory[] valueProviderFactories, RTreePolicyType rtreePolicyType,
-            ILinearizeComparatorFactory linearizeCmpFactory, int startIODeviceIndex) throws HyracksDataException {
+    protected ITreeIndex createLSMTree(IVirtualBufferCache virtualBufferCache, 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.createLSMTree(virtualBufferCache, ioManager, file, diskBufferCache,
-                    diskFileMapProvider, typeTraits, rtreeCmpFactories, btreeCmpFactories, valueProviderFactories,
-                    rtreePolicyType, bloomFilterFalsePositiveRate, mergePolicy, opTracker, ioScheduler,
-                    ioOpCallbackProvider, linearizeCmpFactory,
-                    opDesc.getFileSplitProvider().getFileSplits()[partition].getIODeviceId());
+            return LSMRTreeUtils.createLSMTree(virtualBufferCache, file, diskBufferCache, diskFileMapProvider,
+                    typeTraits, rtreeCmpFactories, btreeCmpFactories, valueProviderFactories, rtreePolicyType,
+                    bloomFilterFalsePositiveRate, mergePolicy, opTracker, ioScheduler, ioOpCallbackProvider,
+                    linearizeCmpFactory);
         } catch (TreeIndexException e) {
             throw new HyracksDataException(e);
         }
diff --git a/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeWithAntiMatterTuplesDataflowHelper.java b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeWithAntiMatterTuplesDataflowHelper.java
index 3163e3a..ea1b1e2 100644
--- a/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeWithAntiMatterTuplesDataflowHelper.java
+++ b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeWithAntiMatterTuplesDataflowHelper.java
@@ -21,7 +21,6 @@
 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.api.io.IIOManager;
 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;
@@ -49,17 +48,16 @@
     }
 
     @Override
-    protected ITreeIndex createLSMTree(IVirtualBufferCache virtualBufferCache, IIOManager ioManager,
-            FileReference file, IBufferCache diskBufferCache, IFileMapProvider diskFileMapProvider,
-            ITypeTraits[] typeTraits, IBinaryComparatorFactory[] rtreeCmpFactories,
-            IBinaryComparatorFactory[] btreeCmpFactories, ILSMOperationTracker opTracker,
-            IPrimitiveValueProviderFactory[] valueProviderFactories, RTreePolicyType rtreePolicyType,
-            ILinearizeComparatorFactory linearizeCmpFactory, int startIODeviceIndex) throws HyracksDataException {
+    protected ITreeIndex createLSMTree(IVirtualBufferCache virtualBufferCache, 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.createLSMTreeWithAntiMatterTuples(virtualBufferCache, ioManager, file,
-                    diskBufferCache, diskFileMapProvider, typeTraits, rtreeCmpFactories, btreeCmpFactories,
-                    valueProviderFactories, rtreePolicyType, mergePolicy, opTracker, ioScheduler, ioOpCallbackProvider,
-                    linearizeCmpFactory, startIODeviceIndex);
+            return LSMRTreeUtils.createLSMTreeWithAntiMatterTuples(virtualBufferCache, file, diskBufferCache,
+                    diskFileMapProvider, typeTraits, rtreeCmpFactories, btreeCmpFactories, valueProviderFactories,
+                    rtreePolicyType, mergePolicy, opTracker, ioScheduler, ioOpCallbackProvider, linearizeCmpFactory);
         } catch (TreeIndexException e) {
             throw new HyracksDataException(e);
         }
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 632d462..e3e690b 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
@@ -185,6 +185,7 @@
 
         mutableComponent.getRTree().clear();
         mutableComponent.getBTree().clear();
+        mutableComponent.reset();
     }
 
     @Override
@@ -346,11 +347,7 @@
     }
 
     public boolean isEmptyIndex() throws HyracksDataException {
-        return componentsRef.get().isEmpty()
-                && mutableComponent.getBTree().isEmptyTree(
-                        mutableComponent.getBTree().getInteriorFrameFactory().createFrame())
-                && mutableComponent.getRTree().isEmptyTree(
-                        mutableComponent.getRTree().getInteriorFrameFactory().createFrame());
+        return componentsRef.get().isEmpty() && !mutableComponent.isModified();
     }
 
     @Override
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 2013d74..0b71373 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
@@ -263,7 +263,7 @@
         if (!isEmpty) {
             rTreeTupleSorter.sort();
 
-            rTreeBulkloader = diskRTree.createBulkLoader(1.0f, false, 0L);
+            rTreeBulkloader = diskRTree.createBulkLoader(1.0f, false, 0L, false);
             cursor = rTreeTupleSorter;
 
             try {
@@ -305,7 +305,7 @@
             BTree diskBTree = component.getBTree();
 
             // BulkLoad the tuples from the in-memory tree into the new disk BTree.
-            IIndexBulkLoader bTreeBulkloader = diskBTree.createBulkLoader(1.0f, false, numBTreeTuples);
+            IIndexBulkLoader bTreeBulkloader = diskBTree.createBulkLoader(1.0f, false, numBTreeTuples, false);
             IIndexBulkLoader builder = component.getBloomFilter().createBuilder(numBTreeTuples,
                     bloomFilterSpec.getNumHashes(), bloomFilterSpec.getNumBucketsPerElements());
             // scan the memory BTree
@@ -356,7 +356,7 @@
 
         LSMRTreeImmutableComponent mergedComponent = createDiskComponent(componentFactory,
                 mergeOp.getRTreeMergeTarget(), mergeOp.getBTreeMergeTarget(), mergeOp.getBloomFilterMergeTarget(), true);
-        IIndexBulkLoader bulkLoader = mergedComponent.getRTree().createBulkLoader(1.0f, false, 0L);
+        IIndexBulkLoader bulkLoader = mergedComponent.getRTree().createBulkLoader(1.0f, false, 0L, false);
 
         try {
             while (cursor.hasNext()) {
@@ -400,59 +400,72 @@
     }
 
     @Override
-    public IIndexBulkLoader createBulkLoader(float fillLevel, boolean verifyInput, long numElementsHint)
-            throws TreeIndexException {
-        return new LSMRTreeBulkLoader(fillLevel, verifyInput, numElementsHint);
+    public IIndexBulkLoader createBulkLoader(float fillLevel, boolean verifyInput, long numElementsHint,
+            boolean checkIfEmptyIndex) throws TreeIndexException {
+        try {
+            return new LSMRTreeBulkLoader(fillLevel, verifyInput, numElementsHint, checkIfEmptyIndex);
+        } catch (HyracksDataException e) {
+            throw new TreeIndexException(e);
+        }
     }
 
     public class LSMRTreeBulkLoader implements IIndexBulkLoader {
         private final ILSMComponent component;
         private final IIndexBulkLoader bulkLoader;
+        private boolean cleanedUpArtifacts = false;
+        private boolean isEmptyComponent = true;
 
-        public LSMRTreeBulkLoader(float fillFactor, boolean verifyInput, long numElementsHint)
-                throws TreeIndexException {
+        public LSMRTreeBulkLoader(float fillFactor, boolean verifyInput, long numElementsHint, boolean checkIfEmptyIndex)
+                throws TreeIndexException, HyracksDataException {
+            if (checkIfEmptyIndex && !isEmptyIndex()) {
+                throw new TreeIndexException("Cannot load an index that is not empty");
+            }
             // Note that by using a flush target file name, we state that the
             // new bulk loaded tree is "newer" than any other merged tree.
             try {
                 component = createBulkLoadTarget();
-            } catch (HyracksDataException e) {
-                throw new TreeIndexException(e);
-            } catch (IndexException e) {
+            } catch (HyracksDataException | IndexException e) {
                 throw new TreeIndexException(e);
             }
             bulkLoader = ((LSMRTreeImmutableComponent) component).getRTree().createBulkLoader(fillFactor, verifyInput,
-                    numElementsHint);
+                    numElementsHint, false);
         }
 
         @Override
         public void add(ITupleReference tuple) throws HyracksDataException, IndexException {
             try {
                 bulkLoader.add(tuple);
-            } catch (IndexException e) {
-                handleException();
+            } catch (IndexException | HyracksDataException | RuntimeException e) {
+                cleanupArtifacts();
                 throw e;
-            } catch (HyracksDataException e) {
-                handleException();
-                throw e;
-            } catch (RuntimeException e) {
-                handleException();
-                throw e;
+            }
+            if (isEmptyComponent) {
+                isEmptyComponent = false;
             }
         }
 
         @Override
         public void end() throws HyracksDataException, IndexException {
-            bulkLoader.end();
-            lsmHarness.addBulkLoadedComponent(component);
+            if (!cleanedUpArtifacts) {
+                bulkLoader.end();
+                if (isEmptyComponent) {
+                    cleanupArtifacts();
+                } else {
+                    lsmHarness.addBulkLoadedComponent(component);
+                }
+            }
         }
 
-        protected void handleException() throws HyracksDataException {
-            ((LSMRTreeImmutableComponent) component).getRTree().deactivate();
-            ((LSMRTreeImmutableComponent) component).getRTree().destroy();
-            ((LSMRTreeImmutableComponent) component).getBTree().deactivate();
-            ((LSMRTreeImmutableComponent) component).getBTree().destroy();
-            ((LSMRTreeImmutableComponent) component).getBloomFilter().deactivate();
-            ((LSMRTreeImmutableComponent) component).getBloomFilter().destroy();
+        protected void cleanupArtifacts() throws HyracksDataException {
+            if (!cleanedUpArtifacts) {
+                cleanedUpArtifacts = true;
+                ((LSMRTreeImmutableComponent) component).getRTree().deactivate();
+                ((LSMRTreeImmutableComponent) component).getRTree().destroy();
+                ((LSMRTreeImmutableComponent) component).getBTree().deactivate();
+                ((LSMRTreeImmutableComponent) component).getBTree().destroy();
+                ((LSMRTreeImmutableComponent) component).getBloomFilter().deactivate();
+                ((LSMRTreeImmutableComponent) component).getBloomFilter().destroy();
+            }
         }
     }
 
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 cfdf706..9502d5e 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
@@ -26,7 +26,6 @@
 
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.api.io.FileReference;
-import edu.uci.ics.hyracks.api.io.IIOManager;
 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;
@@ -53,10 +52,9 @@
         }
     };
 
-    public LSMRTreeFileManager(IIOManager ioManager, IFileMapProvider fileMapProvider, FileReference file,
-            TreeIndexFactory<? extends ITreeIndex> rtreeFactory, TreeIndexFactory<? extends ITreeIndex> btreeFactory,
-            int ioDeviceId) {
-        super(ioManager, fileMapProvider, file, null, ioDeviceId);
+    public LSMRTreeFileManager(IFileMapProvider fileMapProvider, FileReference file,
+            TreeIndexFactory<? extends ITreeIndex> rtreeFactory, TreeIndexFactory<? extends ITreeIndex> btreeFactory) {
+        super(fileMapProvider, file, null);
         this.rtreeFactory = rtreeFactory;
         this.btreeFactory = btreeFactory;
     }
@@ -92,15 +90,15 @@
         ArrayList<ComparableFileName> allBTreeFiles = new ArrayList<ComparableFileName>();
         ArrayList<ComparableFileName> allBloomFilterFiles = new ArrayList<ComparableFileName>();
 
-        // Gather files from the IODeviceHandle.
-        cleanupAndGetValidFilesInternal(dev, btreeFilter, btreeFactory, allBTreeFiles);
+        // Gather files.
+        cleanupAndGetValidFilesInternal(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(dev, btreeFilesSet, allRTreeFiles, rtreeFilter, rtreeFactory);
-        validateFiles(dev, btreeFilesSet, allBloomFilterFiles, bloomFilterFilter, null);
+        validateFiles(btreeFilesSet, allRTreeFiles, rtreeFilter, rtreeFactory);
+        validateFiles(btreeFilesSet, allBloomFilterFiles, bloomFilterFilter, null);
 
         // Sanity check.
         if (allRTreeFiles.size() != allBTreeFiles.size() || allBTreeFiles.size() != allBloomFilterFiles.size()) {
diff --git a/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTreeWithAntiMatterTuples.java b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTreeWithAntiMatterTuples.java
index cf3bbc6e..0be497a 100644
--- a/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTreeWithAntiMatterTuples.java
+++ b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTreeWithAntiMatterTuples.java
@@ -273,7 +273,7 @@
             bTreeTupleSorter.sort();
         }
 
-        IIndexBulkLoader rTreeBulkloader = diskRTree.createBulkLoader(1.0f, false, 0L);
+        IIndexBulkLoader rTreeBulkloader = diskRTree.createBulkLoader(1.0f, false, 0L, false);
         LSMRTreeWithAntiMatterTuplesFlushCursor cursor = new LSMRTreeWithAntiMatterTuplesFlushCursor(rTreeTupleSorter,
                 bTreeTupleSorter, comparatorFields, linearizerArray);
         cursor.open(null, null);
@@ -326,7 +326,7 @@
         LSMRTreeImmutableComponent component = createDiskComponent(componentFactory, mergeOp.getRTreeMergeTarget(),
                 null, null, true);
         RTree mergedRTree = component.getRTree();
-        IIndexBulkLoader bulkloader = mergedRTree.createBulkLoader(1.0f, false, 0L);
+        IIndexBulkLoader bulkloader = mergedRTree.createBulkLoader(1.0f, false, 0L, false);
         try {
             while (cursor.hasNext()) {
                 cursor.next();
@@ -363,9 +363,14 @@
     }
 
     @Override
-    public IIndexBulkLoader createBulkLoader(float fillLevel, boolean verifyInput, long numElementsHint)
-            throws TreeIndexException {
-        return new LSMRTreeWithAntiMatterTuplesBulkLoader(fillLevel, verifyInput, numElementsHint);
+    public IIndexBulkLoader createBulkLoader(float fillLevel, boolean verifyInput, long numElementsHint,
+            boolean checkIfEmptyIndex) throws TreeIndexException {
+        try {
+            return new LSMRTreeWithAntiMatterTuplesBulkLoader(fillLevel, verifyInput, numElementsHint,
+                    checkIfEmptyIndex);
+        } catch (HyracksDataException e) {
+            throw new TreeIndexException(e);
+        }
     }
 
     private ILSMComponent createBulkLoadTarget() throws HyracksDataException, IndexException {
@@ -377,47 +382,56 @@
     public class LSMRTreeWithAntiMatterTuplesBulkLoader implements IIndexBulkLoader {
         private final ILSMComponent component;
         private final IIndexBulkLoader bulkLoader;
+        private boolean cleanedUpArtifacts = false;
+        private boolean isEmptyComponent = true;
 
-        public LSMRTreeWithAntiMatterTuplesBulkLoader(float fillFactor, boolean verifyInput, long numElementsHint)
-                throws TreeIndexException {
+        public LSMRTreeWithAntiMatterTuplesBulkLoader(float fillFactor, boolean verifyInput, long numElementsHint,
+                boolean checkIfEmptyIndex) throws TreeIndexException, HyracksDataException {
+            if (checkIfEmptyIndex && !isEmptyIndex()) {
+                throw new TreeIndexException("Cannot load an index that is not empty");
+            }
             // Note that by using a flush target file name, we state that the
             // new bulk loaded tree is "newer" than any other merged tree.
             try {
                 component = createBulkLoadTarget();
-            } catch (HyracksDataException e) {
-                throw new TreeIndexException(e);
-            } catch (IndexException e) {
+            } catch (HyracksDataException | IndexException e) {
                 throw new TreeIndexException(e);
             }
             bulkLoader = ((LSMRTreeImmutableComponent) component).getRTree().createBulkLoader(fillFactor, verifyInput,
-                    numElementsHint);
+                    numElementsHint, false);
         }
 
         @Override
         public void add(ITupleReference tuple) throws HyracksDataException, IndexException {
             try {
                 bulkLoader.add(tuple);
-            } catch (IndexException e) {
-                handleException();
+            } catch (IndexException | HyracksDataException | RuntimeException e) {
+                cleanupArtifacts();
                 throw e;
-            } catch (HyracksDataException e) {
-                handleException();
-                throw e;
-            } catch (RuntimeException e) {
-                handleException();
-                throw e;
+            }
+            if (isEmptyComponent) {
+                isEmptyComponent = false;
             }
         }
 
         @Override
         public void end() throws HyracksDataException, IndexException {
-            bulkLoader.end();
-            lsmHarness.addBulkLoadedComponent(component);
+            if (!cleanedUpArtifacts) {
+                bulkLoader.end();
+                if (isEmptyComponent) {
+                    cleanupArtifacts();
+                } else {
+                    lsmHarness.addBulkLoadedComponent(component);
+                }
+            }
         }
 
-        protected void handleException() throws HyracksDataException {
-            ((LSMRTreeImmutableComponent) component).getRTree().deactivate();
-            ((LSMRTreeImmutableComponent) component).getRTree().destroy();
+        protected void cleanupArtifacts() throws HyracksDataException {
+            if (!cleanedUpArtifacts) {
+                cleanedUpArtifacts = true;
+                ((LSMRTreeImmutableComponent) component).getRTree().deactivate();
+                ((LSMRTreeImmutableComponent) component).getRTree().destroy();
+            }
         }
 
     }
diff --git a/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTreeWithAntiMatterTuplesFileManager.java b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTreeWithAntiMatterTuplesFileManager.java
index f426dea..8ef143f 100644
--- a/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTreeWithAntiMatterTuplesFileManager.java
+++ b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTreeWithAntiMatterTuplesFileManager.java
@@ -16,7 +16,6 @@
 package edu.uci.ics.hyracks.storage.am.lsm.rtree.impls;
 
 import edu.uci.ics.hyracks.api.io.FileReference;
-import edu.uci.ics.hyracks.api.io.IIOManager;
 import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
 import edu.uci.ics.hyracks.storage.am.lsm.common.impls.AbstractLSMIndexFileManager;
 import edu.uci.ics.hyracks.storage.am.lsm.common.impls.TreeIndexFactory;
@@ -24,8 +23,8 @@
 
 public class LSMRTreeWithAntiMatterTuplesFileManager extends AbstractLSMIndexFileManager {
 
-    public LSMRTreeWithAntiMatterTuplesFileManager(IIOManager ioManager, IFileMapProvider fileMapProvider,
-            FileReference file, TreeIndexFactory<? extends ITreeIndex> rtreeFactory, int ioDeviceId) {
-        super(ioManager, fileMapProvider, file, rtreeFactory, ioDeviceId);
+    public LSMRTreeWithAntiMatterTuplesFileManager(IFileMapProvider fileMapProvider, FileReference file,
+            TreeIndexFactory<? extends ITreeIndex> rtreeFactory) {
+        super(fileMapProvider, file, rtreeFactory);
     }
 }
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 07c6815..50e5d09 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
@@ -19,7 +19,6 @@
 import edu.uci.ics.hyracks.api.dataflow.value.ILinearizeComparatorFactory;
 import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
 import edu.uci.ics.hyracks.api.io.FileReference;
-import edu.uci.ics.hyracks.api.io.IIOManager;
 import edu.uci.ics.hyracks.data.std.primitive.DoublePointable;
 import edu.uci.ics.hyracks.data.std.primitive.IntegerPointable;
 import edu.uci.ics.hyracks.storage.am.bloomfilter.impls.BloomFilterFactory;
@@ -59,28 +58,13 @@
 import edu.uci.ics.hyracks.storage.common.file.IFileMapProvider;
 
 public class LSMRTreeUtils {
-    public static LSMRTree createLSMTree(IVirtualBufferCache virtualBufferCache, IIOManager ioManager,
-            FileReference file, IBufferCache diskBufferCache, IFileMapProvider diskFileMapProvider,
-            ITypeTraits[] typeTraits, IBinaryComparatorFactory[] rtreeCmpFactories,
-            IBinaryComparatorFactory[] btreeCmpFactories, IPrimitiveValueProviderFactory[] valueProviderFactories,
-            RTreePolicyType rtreePolicyType, double bloomFilterFalsePositiveRate, ILSMMergePolicy mergePolicy,
-            ILSMOperationTracker opTracker, ILSMIOOperationScheduler ioScheduler,
-            ILSMIOOperationCallbackProvider ioOpCallbackProvider, ILinearizeComparatorFactory linearizeCmpFactory)
-            throws TreeIndexException {
-        return createLSMTree(virtualBufferCache, ioManager, file, diskBufferCache, diskFileMapProvider, typeTraits,
-                rtreeCmpFactories, btreeCmpFactories, valueProviderFactories, rtreePolicyType,
-                bloomFilterFalsePositiveRate, mergePolicy, opTracker, ioScheduler, ioOpCallbackProvider,
-                linearizeCmpFactory, 0);
-    }
-
-    public static LSMRTree createLSMTree(IVirtualBufferCache virtualBufferCache, IIOManager ioManager,
-            FileReference file, IBufferCache diskBufferCache, IFileMapProvider diskFileMapProvider,
-            ITypeTraits[] typeTraits, IBinaryComparatorFactory[] rtreeCmpFactories,
-            IBinaryComparatorFactory[] btreeCmpFactories, IPrimitiveValueProviderFactory[] valueProviderFactories,
-            RTreePolicyType rtreePolicyType, double bloomFilterFalsePositiveRate, ILSMMergePolicy mergePolicy,
-            ILSMOperationTracker opTracker, ILSMIOOperationScheduler ioScheduler,
-            ILSMIOOperationCallbackProvider ioOpCallbackProvider, ILinearizeComparatorFactory linearizeCmpFactory,
-            int ioDeviceId) throws TreeIndexException {
+    public static LSMRTree createLSMTree(IVirtualBufferCache virtualBufferCache, FileReference file,
+            IBufferCache diskBufferCache, IFileMapProvider diskFileMapProvider, ITypeTraits[] typeTraits,
+            IBinaryComparatorFactory[] rtreeCmpFactories, IBinaryComparatorFactory[] btreeCmpFactories,
+            IPrimitiveValueProviderFactory[] valueProviderFactories, RTreePolicyType rtreePolicyType,
+            double bloomFilterFalsePositiveRate, ILSMMergePolicy mergePolicy, ILSMOperationTracker opTracker,
+            ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallbackProvider ioOpCallbackProvider,
+            ILinearizeComparatorFactory linearizeCmpFactory) throws TreeIndexException {
         LSMTypeAwareTupleWriterFactory rtreeTupleWriterFactory = new LSMTypeAwareTupleWriterFactory(typeTraits, false);
         LSMTypeAwareTupleWriterFactory btreeTupleWriterFactory = new LSMTypeAwareTupleWriterFactory(typeTraits, true);
 
@@ -113,8 +97,8 @@
         BloomFilterFactory bloomFilterFactory = new BloomFilterFactory(diskBufferCache, diskFileMapProvider,
                 bloomFilterKeyFields);
 
-        ILSMIndexFileManager fileNameManager = new LSMRTreeFileManager(ioManager, diskFileMapProvider, file,
-                diskRTreeFactory, diskBTreeFactory, ioDeviceId);
+        ILSMIndexFileManager fileNameManager = new LSMRTreeFileManager(diskFileMapProvider, file, diskRTreeFactory,
+                diskBTreeFactory);
         LSMRTree lsmTree = new LSMRTree(virtualBufferCache, rtreeInteriorFrameFactory, rtreeLeafFrameFactory,
                 btreeInteriorFrameFactory, btreeLeafFrameFactory, fileNameManager, diskRTreeFactory, diskBTreeFactory,
                 bloomFilterFactory, bloomFilterFalsePositiveRate, diskFileMapProvider, typeTraits.length,
@@ -124,26 +108,13 @@
     }
 
     public static LSMRTreeWithAntiMatterTuples createLSMTreeWithAntiMatterTuples(
-            IVirtualBufferCache virtualBufferCache, IIOManager ioManager, FileReference file,
-            IBufferCache diskBufferCache, IFileMapProvider diskFileMapProvider, ITypeTraits[] typeTraits,
+            IVirtualBufferCache virtualBufferCache, FileReference file, IBufferCache diskBufferCache,
+            IFileMapProvider diskFileMapProvider, ITypeTraits[] typeTraits,
             IBinaryComparatorFactory[] rtreeCmpFactories, IBinaryComparatorFactory[] btreeCmpFactories,
             IPrimitiveValueProviderFactory[] valueProviderFactories, RTreePolicyType rtreePolicyType,
             ILSMMergePolicy mergePolicy, ILSMOperationTracker opTracker, ILSMIOOperationScheduler ioScheduler,
             ILSMIOOperationCallbackProvider ioOpCallbackProvider, ILinearizeComparatorFactory linearizerCmpFactory)
             throws TreeIndexException {
-        return createLSMTreeWithAntiMatterTuples(virtualBufferCache, ioManager, file, diskBufferCache,
-                diskFileMapProvider, typeTraits, rtreeCmpFactories, btreeCmpFactories, valueProviderFactories,
-                rtreePolicyType, mergePolicy, opTracker, ioScheduler, ioOpCallbackProvider, linearizerCmpFactory, 0);
-    }
-
-    public static LSMRTreeWithAntiMatterTuples createLSMTreeWithAntiMatterTuples(
-            IVirtualBufferCache virtualBufferCache, IIOManager ioManager, FileReference file,
-            IBufferCache diskBufferCache, IFileMapProvider diskFileMapProvider, ITypeTraits[] typeTraits,
-            IBinaryComparatorFactory[] rtreeCmpFactories, IBinaryComparatorFactory[] btreeCmpFactories,
-            IPrimitiveValueProviderFactory[] valueProviderFactories, RTreePolicyType rtreePolicyType,
-            ILSMMergePolicy mergePolicy, ILSMOperationTracker opTracker, ILSMIOOperationScheduler ioScheduler,
-            ILSMIOOperationCallbackProvider ioOpCallbackProvider, ILinearizeComparatorFactory linearizerCmpFactory,
-            int ioDeviceId) throws TreeIndexException {
         LSMRTreeTupleWriterFactory rtreeTupleWriterFactory = new LSMRTreeTupleWriterFactory(typeTraits, false);
         LSMRTreeTupleWriterFactory btreeTupleWriterFactory = new LSMRTreeTupleWriterFactory(typeTraits, true);
 
@@ -178,8 +149,8 @@
         IBinaryComparatorFactory[] linearizerArray = { linearizerCmpFactory,
                 btreeCmpFactories[btreeCmpFactories.length - 1] };
 
-        ILSMIndexFileManager fileNameManager = new LSMRTreeWithAntiMatterTuplesFileManager(ioManager,
-                diskFileMapProvider, file, diskRTreeFactory, ioDeviceId);
+        ILSMIndexFileManager fileNameManager = new LSMRTreeWithAntiMatterTuplesFileManager(diskFileMapProvider, file,
+                diskRTreeFactory);
         LSMRTreeWithAntiMatterTuples lsmTree = new LSMRTreeWithAntiMatterTuples(virtualBufferCache,
                 rtreeInteriorFrameFactory, rtreeLeafFrameFactory, btreeInteriorFrameFactory, btreeLeafFrameFactory,
                 fileNameManager, diskRTreeFactory, bulkLoadRTreeFactory, diskFileMapProvider, typeTraits.length,
diff --git a/hyracks/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/impls/RTree.java b/hyracks/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/impls/RTree.java
index 5454832..528c004 100644
--- a/hyracks/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/impls/RTree.java
+++ b/hyracks/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/impls/RTree.java
@@ -860,8 +860,8 @@
     }
 
     @Override
-    public IIndexBulkLoader createBulkLoader(float fillFactor, boolean verifyInput, long numElementsHint)
-            throws TreeIndexException {
+    public IIndexBulkLoader createBulkLoader(float fillFactor, boolean verifyInput, long numElementsHint,
+            boolean checkIfEmptyIndex) throws TreeIndexException {
         // TODO: verifyInput currently does nothing.
         try {
             return new RTreeBulkLoader(fillFactor);
diff --git a/hyracks/hyracks-storage-common/src/main/java/edu/uci/ics/hyracks/storage/common/file/ILocalResourceRepository.java b/hyracks/hyracks-storage-common/src/main/java/edu/uci/ics/hyracks/storage/common/file/ILocalResourceRepository.java
index e07a375..7ae2414 100644
--- a/hyracks/hyracks-storage-common/src/main/java/edu/uci/ics/hyracks/storage/common/file/ILocalResourceRepository.java
+++ b/hyracks/hyracks-storage-common/src/main/java/edu/uci/ics/hyracks/storage/common/file/ILocalResourceRepository.java
@@ -24,11 +24,11 @@
 
     public LocalResource getResourceByName(String name) throws HyracksDataException;
 
-    public void insert(LocalResource resource, int ioDeviceId) throws HyracksDataException;
+    public void insert(LocalResource resource) throws HyracksDataException;
 
-    public void deleteResourceById(long id, int ioDeviceId) throws HyracksDataException;
+    public void deleteResourceById(long id) throws HyracksDataException;
 
-    public void deleteResourceByName(String name, int ioDeviceId) throws HyracksDataException;
+    public void deleteResourceByName(String name) throws HyracksDataException;
 
     public List<LocalResource> getAllResources() throws HyracksDataException;
 }
diff --git a/hyracks/hyracks-storage-common/src/main/java/edu/uci/ics/hyracks/storage/common/file/TransientLocalResourceRepository.java b/hyracks/hyracks-storage-common/src/main/java/edu/uci/ics/hyracks/storage/common/file/TransientLocalResourceRepository.java
index f4fa440..63b71a3 100644
--- a/hyracks/hyracks-storage-common/src/main/java/edu/uci/ics/hyracks/storage/common/file/TransientLocalResourceRepository.java
+++ b/hyracks/hyracks-storage-common/src/main/java/edu/uci/ics/hyracks/storage/common/file/TransientLocalResourceRepository.java
@@ -37,7 +37,7 @@
     }
 
     @Override
-    public synchronized void insert(LocalResource resource, int ioDeviceId) throws HyracksDataException {
+    public synchronized void insert(LocalResource resource) throws HyracksDataException {
         long id = resource.getResourceId();
 
         if (id2ResourceMap.containsKey(id)) {
@@ -48,7 +48,7 @@
     }
 
     @Override
-    public synchronized void deleteResourceById(long id, int ioDeviceId) throws HyracksDataException {
+    public synchronized void deleteResourceById(long id) throws HyracksDataException {
         LocalResource resource = id2ResourceMap.get(id);
         if (resource == null) {
             throw new HyracksDataException("Resource doesn't exist");
@@ -58,7 +58,7 @@
     }
 
     @Override
-    public synchronized void deleteResourceByName(String name, int ioDeviceId) throws HyracksDataException {
+    public synchronized void deleteResourceByName(String name) throws HyracksDataException {
         LocalResource resource = name2ResourceMap.get(name);
         if (resource == null) {
             throw new HyracksDataException("Resource doesn't exist");
diff --git a/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/btree/AbstractSearchOperationCallbackTest.java b/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/btree/AbstractSearchOperationCallbackTest.java
index 5bc7f28..02ce7f1 100644
--- a/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/btree/AbstractSearchOperationCallbackTest.java
+++ b/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/btree/AbstractSearchOperationCallbackTest.java
@@ -85,7 +85,7 @@
         private int blockingValue;
         private int expectedAfterBlock;
 
-        public SearchTask() {
+        public SearchTask() throws HyracksDataException {
             this.cb = new SynchronizingSearchOperationCallback();
             this.accessor = index.createAccessor(NoOpOperationCallback.INSTANCE, cb);
             this.cursor = accessor.createSearchCursor();
@@ -199,7 +199,7 @@
         private final ArrayTupleBuilder builder;
         private final ArrayTupleReference tuple;
 
-        public InsertionTask() {
+        public InsertionTask() throws HyracksDataException {
             this.accessor = index.createAccessor(NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE);
             this.builder = new ArrayTupleBuilder(NUM_KEY_FIELDS);
             this.tuple = new ArrayTupleReference();
diff --git a/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/btree/OrderedIndexBulkLoadTest.java b/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/btree/OrderedIndexBulkLoadTest.java
index f36419c..54eac1e 100644
--- a/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/btree/OrderedIndexBulkLoadTest.java
+++ b/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/btree/OrderedIndexBulkLoadTest.java
@@ -62,9 +62,9 @@
         ctx.getIndex().deactivate();
         ctx.getIndex().destroy();
     }
-    
+
     @Override
     protected String getTestOpName() {
         return "BulkLoad";
     }
-}
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/btree/OrderedIndexExamplesTest.java b/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/btree/OrderedIndexExamplesTest.java
index 5b8f7d9..eafeff2 100644
--- a/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/btree/OrderedIndexExamplesTest.java
+++ b/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/btree/OrderedIndexExamplesTest.java
@@ -621,7 +621,7 @@
             LOGGER.info("Bulk loading " + ins + " tuples");
         }
         long start = System.currentTimeMillis();
-        IIndexBulkLoader bulkLoader = treeIndex.createBulkLoader(0.7f, false, ins);
+        IIndexBulkLoader bulkLoader = treeIndex.createBulkLoader(0.7f, false, ins, true);
         ArrayTupleBuilder tb = new ArrayTupleBuilder(fieldCount);
         ArrayTupleReference tuple = new ArrayTupleReference();
         for (int i = 0; i < ins; i++) {
@@ -691,7 +691,7 @@
             treeIndex.activate();
 
             // Load sorted records, and expect to fail at tuple i.
-            IIndexBulkLoader bulkLoader = treeIndex.createBulkLoader(0.7f, true, ins);
+            IIndexBulkLoader bulkLoader = treeIndex.createBulkLoader(0.7f, true, ins, true);
             for (int j = 0; j < ins; j++) {
                 if (j > i) {
                     fail("Bulk load failure test unexpectedly succeeded past tuple: " + j);
diff --git a/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/btree/OrderedIndexTestContext.java b/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/btree/OrderedIndexTestContext.java
index 62fd98b..85c7fa5 100644
--- a/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/btree/OrderedIndexTestContext.java
+++ b/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/btree/OrderedIndexTestContext.java
@@ -19,6 +19,7 @@
 import java.util.TreeSet;
 
 import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.storage.am.common.CheckTuple;
 import edu.uci.ics.hyracks.storage.am.common.IndexTestContext;
 import edu.uci.ics.hyracks.storage.am.common.api.IIndex;
@@ -28,17 +29,17 @@
 
     protected final TreeSet<CheckTuple> checkTuples = new TreeSet<CheckTuple>();
 
-    public OrderedIndexTestContext(ISerializerDeserializer[] fieldSerdes, IIndex index) {
+    public OrderedIndexTestContext(ISerializerDeserializer[] fieldSerdes, IIndex index) throws HyracksDataException {
         super(fieldSerdes, index);
     }
 
     public void upsertCheckTuple(CheckTuple checkTuple, Collection<CheckTuple> checkTuples) {
-    	if (checkTuples.contains(checkTuple)) {
+        if (checkTuples.contains(checkTuple)) {
             checkTuples.remove(checkTuple);
         }
         checkTuples.add(checkTuple);
     }
-    
+
     @Override
     public TreeSet<CheckTuple> getCheckTuples() {
         return checkTuples;
diff --git a/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/common/AbstractIndexTestWorker.java b/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/common/AbstractIndexTestWorker.java
index 9f38d3a..3c8b740 100644
--- a/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/common/AbstractIndexTestWorker.java
+++ b/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/common/AbstractIndexTestWorker.java
@@ -35,7 +35,8 @@
 
     protected final IIndexAccessor indexAccessor;
 
-    public AbstractIndexTestWorker(DataGenThread dataGen, TestOperationSelector opSelector, IIndex index, int numBatches) {
+    public AbstractIndexTestWorker(DataGenThread dataGen, TestOperationSelector opSelector, IIndex index, int numBatches)
+            throws HyracksDataException {
         this.dataGen = dataGen;
         this.opSelector = opSelector;
         this.numBatches = numBatches;
diff --git a/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/common/IIndexTestWorkerFactory.java b/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/common/IIndexTestWorkerFactory.java
index f56614e..65ed2ce 100644
--- a/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/common/IIndexTestWorkerFactory.java
+++ b/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/common/IIndexTestWorkerFactory.java
@@ -15,10 +15,11 @@
 
 package edu.uci.ics.hyracks.storage.am.common;
 
+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.datagen.DataGenThread;
 
 public interface IIndexTestWorkerFactory {
     public AbstractIndexTestWorker create(DataGenThread dataGen, TestOperationSelector opSelector, IIndex index,
-            int numBatches);
+            int numBatches) throws HyracksDataException;
 }
diff --git a/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/common/IndexMultiThreadTestDriver.java b/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/common/IndexMultiThreadTestDriver.java
index 0cb36d6..d8f9485 100644
--- a/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/common/IndexMultiThreadTestDriver.java
+++ b/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/common/IndexMultiThreadTestDriver.java
@@ -46,7 +46,7 @@
     }
 
     public long[] run(int numThreads, int numRepeats, int numOps, int batchSize) throws InterruptedException,
-            TreeIndexException {
+            TreeIndexException, HyracksDataException {
         int numBatches = numOps / batchSize;
         int threadNumBatches = numBatches / numThreads;
         if (threadNumBatches <= 0) {
diff --git a/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/common/IndexTestContext.java b/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/common/IndexTestContext.java
index f91b1bf..339cd14 100644
--- a/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/common/IndexTestContext.java
+++ b/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/common/IndexTestContext.java
@@ -18,6 +18,7 @@
 import java.util.Collection;
 
 import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleReference;
 import edu.uci.ics.hyracks.storage.am.common.api.IIndex;
@@ -31,7 +32,7 @@
     protected final ArrayTupleReference tuple = new ArrayTupleReference();
     protected final IIndexAccessor indexAccessor;
 
-    public IndexTestContext(ISerializerDeserializer[] fieldSerdes, IIndex index) {
+    public IndexTestContext(ISerializerDeserializer[] fieldSerdes, IIndex index) throws HyracksDataException {
         this.fieldSerdes = fieldSerdes;
         this.index = index;
         this.indexAccessor = (IIndexAccessor) index.createAccessor(TestOperationCallback.INSTANCE,
diff --git a/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/common/TreeIndexTestUtils.java b/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/common/TreeIndexTestUtils.java
index 852a0b6..b5742b5 100644
--- a/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/common/TreeIndexTestUtils.java
+++ b/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/common/TreeIndexTestUtils.java
@@ -243,7 +243,7 @@
         ArrayTupleBuilder tupleBuilder = new ArrayTupleBuilder(fieldCount);
         ArrayTupleReference tuple = new ArrayTupleReference();
         // Perform bulk load.
-        IIndexBulkLoader bulkLoader = ctx.getIndex().createBulkLoader(0.7f, false, numTuples);
+        IIndexBulkLoader bulkLoader = ctx.getIndex().createBulkLoader(0.7f, false, numTuples, false);
         int c = 1;
         for (CheckTuple checkTuple : checkTuples) {
             if (LOGGER.isLoggable(Level.INFO)) {
diff --git a/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/AbstractRTreeBulkLoadTest.java b/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/AbstractRTreeBulkLoadTest.java
index e8d9c96..11adaa2 100644
--- a/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/AbstractRTreeBulkLoadTest.java
+++ b/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/AbstractRTreeBulkLoadTest.java
@@ -26,11 +26,9 @@
 public abstract class AbstractRTreeBulkLoadTest extends AbstractRTreeTestDriver {
 
     private final RTreeTestUtils rTreeTestUtils;
-    private final int bulkLoadRounds;
 
-    public AbstractRTreeBulkLoadTest(int bulkLoadRounds, boolean testRstarPolicy) {
+    public AbstractRTreeBulkLoadTest(boolean testRstarPolicy) {
         super(testRstarPolicy);
-        this.bulkLoadRounds = bulkLoadRounds;
         this.rTreeTestUtils = new RTreeTestUtils();
     }
 
@@ -41,19 +39,19 @@
         AbstractRTreeTestContext ctx = createTestContext(fieldSerdes, valueProviderFactories, numKeys, rtreePolicyType);
         ctx.getIndex().create();
         ctx.getIndex().activate();
-        for (int i = 0; i < bulkLoadRounds; i++) {
-            // We assume all fieldSerdes are of the same type. Check the first
-            // one to determine which field types to generate.
-            if (fieldSerdes[0] instanceof IntegerSerializerDeserializer) {
-                rTreeTestUtils.bulkLoadIntTuples(ctx, numTuplesToInsert, getRandom());
-            } else if (fieldSerdes[0] instanceof DoubleSerializerDeserializer) {
-                rTreeTestUtils.bulkLoadDoubleTuples(ctx, numTuplesToInsert, getRandom());
-            }
 
-            rTreeTestUtils.checkScan(ctx);
-            rTreeTestUtils.checkDiskOrderScan(ctx);
-            rTreeTestUtils.checkRangeSearch(ctx, key);
+        // We assume all fieldSerdes are of the same type. Check the first
+        // one to determine which field types to generate.
+        if (fieldSerdes[0] instanceof IntegerSerializerDeserializer) {
+            rTreeTestUtils.bulkLoadIntTuples(ctx, numTuplesToInsert, getRandom());
+        } else if (fieldSerdes[0] instanceof DoubleSerializerDeserializer) {
+            rTreeTestUtils.bulkLoadDoubleTuples(ctx, numTuplesToInsert, getRandom());
         }
+
+        rTreeTestUtils.checkScan(ctx);
+        rTreeTestUtils.checkDiskOrderScan(ctx);
+        rTreeTestUtils.checkRangeSearch(ctx, key);
+
         ctx.getIndex().deactivate();
         ctx.getIndex().destroy();
     }
diff --git a/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/AbstractRTreeExamplesTest.java b/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/AbstractRTreeExamplesTest.java
index 50ff154..44ab0d9 100644
--- a/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/AbstractRTreeExamplesTest.java
+++ b/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/AbstractRTreeExamplesTest.java
@@ -709,7 +709,7 @@
             LOGGER.info("Bulk loading " + numInserts + " tuples");
         }
         long start = System.currentTimeMillis();
-        IIndexBulkLoader bulkLoader = treeIndex.createBulkLoader(0.7f, false, numInserts);
+        IIndexBulkLoader bulkLoader = treeIndex.createBulkLoader(0.7f, false, numInserts, true);
         ArrayTupleBuilder tb = new ArrayTupleBuilder(fieldCount);
         ArrayTupleReference tuple = new ArrayTupleReference();
 
diff --git a/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/AbstractRTreeTestContext.java b/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/AbstractRTreeTestContext.java
index 135b306..77c0665 100644
--- a/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/AbstractRTreeTestContext.java
+++ b/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/AbstractRTreeTestContext.java
@@ -18,6 +18,7 @@
 import java.util.Collection;
 
 import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.storage.am.common.IndexTestContext;
 import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
 import edu.uci.ics.hyracks.storage.am.common.util.HashMultiSet;
@@ -25,8 +26,9 @@
 @SuppressWarnings("rawtypes")
 public abstract class AbstractRTreeTestContext extends IndexTestContext<RTreeCheckTuple> {
     private final HashMultiSet<RTreeCheckTuple> checkTuples = new HashMultiSet<RTreeCheckTuple>();
-	
-    public AbstractRTreeTestContext(ISerializerDeserializer[] fieldSerdes, ITreeIndex treeIndex) {
+
+    public AbstractRTreeTestContext(ISerializerDeserializer[] fieldSerdes, ITreeIndex treeIndex)
+            throws HyracksDataException {
         super(fieldSerdes, treeIndex);
     }
 
diff --git a/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestNCApplicationContext.java b/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestNCApplicationContext.java
index 756af98..9b77e23 100644
--- a/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestNCApplicationContext.java
+++ b/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestNCApplicationContext.java
@@ -21,6 +21,7 @@
 import edu.uci.ics.hyracks.api.context.IHyracksRootContext;
 import edu.uci.ics.hyracks.api.job.IJobSerializerDeserializerContainer;
 import edu.uci.ics.hyracks.api.messages.IMessageBroker;
+import edu.uci.ics.hyracks.api.resources.memory.IMemoryManager;
 
 public class TestNCApplicationContext implements INCApplicationContext {
     private final IHyracksRootContext rootCtx;
@@ -29,9 +30,32 @@
     private Serializable distributedState;
     private Object appObject;
 
+    private final IMemoryManager mm;
+
     public TestNCApplicationContext(IHyracksRootContext rootCtx, String nodeId) {
         this.rootCtx = rootCtx;
         this.nodeId = nodeId;
+        mm = new IMemoryManager() {
+            @Override
+            public long getMaximumMemory() {
+                return Long.MAX_VALUE;
+            }
+
+            @Override
+            public long getAvailableMemory() {
+                return Long.MAX_VALUE;
+            }
+
+            @Override
+            public void deallocate(long memory) {
+
+            }
+
+            @Override
+            public boolean allocate(long memory) {
+                return true;
+            }
+        };
     }
 
     @Override
@@ -67,14 +91,18 @@
 
     @Override
     public IMessageBroker getMessageBroker() {
-        // TODO Auto-generated method stub
         return null;
     }
 
+	@Override
+	public IJobSerializerDeserializerContainer getJobSerializerDeserializerContainer() {
+		// TODO Auto-generated method stub
+		return null;
+	}
+
     @Override
-    public IJobSerializerDeserializerContainer getJobSerializerDeserializerContainer() {
-        // TODO Auto-generated method stub
-        return null;
+    public IMemoryManager getMemoryManager() {
+        return mm;
     }
 
     @Override
@@ -88,4 +116,4 @@
         // TODO Auto-generated method stub
 
     }
-}
\ No newline at end of file
+}
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/multithread/BTreeTestWorker.java b/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/multithread/BTreeTestWorker.java
index db74413..262e21c 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/multithread/BTreeTestWorker.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/multithread/BTreeTestWorker.java
@@ -40,7 +40,8 @@
     private final ArrayTupleBuilder deleteTb;
     private final ArrayTupleReference deleteTuple = new ArrayTupleReference();
 
-    public BTreeTestWorker(DataGenThread dataGen, TestOperationSelector opSelector, IIndex index, int numBatches) {
+    public BTreeTestWorker(DataGenThread dataGen, TestOperationSelector opSelector, IIndex index, int numBatches)
+            throws HyracksDataException {
         super(dataGen, opSelector, index, numBatches);
         btree = (BTree) index;
         numKeyFields = btree.getComparatorFactories().length;
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/multithread/BTreeTestWorkerFactory.java b/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/multithread/BTreeTestWorkerFactory.java
index 039b1cf..a6d0425 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/multithread/BTreeTestWorkerFactory.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/multithread/BTreeTestWorkerFactory.java
@@ -15,6 +15,7 @@
 
 package edu.uci.ics.hyracks.storage.am.btree.multithread;
 
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.storage.am.common.AbstractIndexTestWorker;
 import edu.uci.ics.hyracks.storage.am.common.IIndexTestWorkerFactory;
 import edu.uci.ics.hyracks.storage.am.common.TestOperationSelector;
@@ -23,8 +24,8 @@
 
 public class BTreeTestWorkerFactory implements IIndexTestWorkerFactory {
     @Override
-    public AbstractIndexTestWorker create(DataGenThread dataGen, TestOperationSelector opSelector,
-            IIndex index, int numBatches) {
+    public AbstractIndexTestWorker create(DataGenThread dataGen, TestOperationSelector opSelector, IIndex index,
+            int numBatches) throws HyracksDataException {
         return new BTreeTestWorker(dataGen, opSelector, index, numBatches);
     }
 }
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/util/BTreeTestContext.java b/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/util/BTreeTestContext.java
index a3b82ad..7aa6f18 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/util/BTreeTestContext.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/util/BTreeTestContext.java
@@ -18,6 +18,7 @@
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
 import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
 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.dataflow.common.util.SerdeUtils;
 import edu.uci.ics.hyracks.storage.am.btree.OrderedIndexTestContext;
@@ -30,7 +31,7 @@
 @SuppressWarnings("rawtypes")
 public class BTreeTestContext extends OrderedIndexTestContext {
 
-    public BTreeTestContext(ISerializerDeserializer[] fieldSerdes, ITreeIndex treeIndex) {
+    public BTreeTestContext(ISerializerDeserializer[] fieldSerdes, ITreeIndex treeIndex) throws HyracksDataException {
         super(fieldSerdes, treeIndex);
     }
 
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeBulkLoadTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeBulkLoadTest.java
index ec4f7e2..e0a2a46 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeBulkLoadTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeBulkLoadTest.java
@@ -51,11 +51,10 @@
     @Override
     protected OrderedIndexTestContext createTestContext(ISerializerDeserializer[] fieldSerdes, int numKeys,
             BTreeLeafFrameType leafType) throws Exception {
-        return LSMBTreeTestContext.create(harness.getVirtualBufferCache(), harness.getIOManager(),
-                harness.getFileReference(), harness.getDiskBufferCache(), harness.getDiskFileMapProvider(),
-                fieldSerdes, numKeys, harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
-                harness.getOperationTracker(), harness.getIOScheduler(), harness.getIOOperationCallbackProvider(),
-                harness.getIODeviceId());
+        return LSMBTreeTestContext.create(harness.getVirtualBufferCache(), harness.getFileReference(),
+                harness.getDiskBufferCache(), harness.getDiskFileMapProvider(), fieldSerdes, numKeys,
+                harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(), harness.getOperationTracker(),
+                harness.getIOScheduler(), harness.getIOOperationCallbackProvider());
     }
 
     @Override
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeDeleteTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeDeleteTest.java
index b294f39..122b788 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeDeleteTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeDeleteTest.java
@@ -51,11 +51,10 @@
     @Override
     protected OrderedIndexTestContext createTestContext(ISerializerDeserializer[] fieldSerdes, int numKeys,
             BTreeLeafFrameType leafType) throws Exception {
-        return LSMBTreeTestContext.create(harness.getVirtualBufferCache(), harness.getIOManager(),
-                harness.getFileReference(), harness.getDiskBufferCache(), harness.getDiskFileMapProvider(),
-                fieldSerdes, numKeys, harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
-                harness.getOperationTracker(), harness.getIOScheduler(), harness.getIOOperationCallbackProvider(),
-                harness.getIODeviceId());
+        return LSMBTreeTestContext.create(harness.getVirtualBufferCache(), harness.getFileReference(),
+                harness.getDiskBufferCache(), harness.getDiskFileMapProvider(), fieldSerdes, numKeys,
+                harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(), harness.getOperationTracker(),
+                harness.getIOScheduler(), harness.getIOOperationCallbackProvider());
     }
 
     @Override
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeExamplesTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeExamplesTest.java
index afdd5cf..60e0492 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeExamplesTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeExamplesTest.java
@@ -34,11 +34,10 @@
     @Override
     protected ITreeIndex createTreeIndex(ITypeTraits[] typeTraits, IBinaryComparatorFactory[] cmpFactories,
             int[] bloomFilterKeyFields) throws TreeIndexException {
-        return LSMBTreeUtils.createLSMTree(harness.getVirtualBufferCache(), harness.getIOManager(),
-                harness.getFileReference(), harness.getDiskBufferCache(), harness.getDiskFileMapProvider(), typeTraits,
-                cmpFactories, bloomFilterKeyFields, harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
-                harness.getOperationTracker(), harness.getIOScheduler(), harness.getIOOperationCallbackProvider(),
-                harness.getIODeviceId());
+        return LSMBTreeUtils.createLSMTree(harness.getVirtualBufferCache(), harness.getFileReference(),
+                harness.getDiskBufferCache(), harness.getDiskFileMapProvider(), typeTraits, cmpFactories,
+                bloomFilterKeyFields, harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
+                harness.getOperationTracker(), harness.getIOScheduler(), harness.getIOOperationCallbackProvider());
     }
 
     @Before
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeInsertTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeInsertTest.java
index ab5fa1f..76a0206 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeInsertTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeInsertTest.java
@@ -51,11 +51,10 @@
     @Override
     protected OrderedIndexTestContext createTestContext(ISerializerDeserializer[] fieldSerdes, int numKeys,
             BTreeLeafFrameType leafType) throws Exception {
-        return LSMBTreeTestContext.create(harness.getVirtualBufferCache(), harness.getIOManager(),
-                harness.getFileReference(), harness.getDiskBufferCache(), harness.getDiskFileMapProvider(),
-                fieldSerdes, numKeys, harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
-                harness.getOperationTracker(), harness.getIOScheduler(), harness.getIOOperationCallbackProvider(),
-                harness.getIODeviceId());
+        return LSMBTreeTestContext.create(harness.getVirtualBufferCache(), harness.getFileReference(),
+                harness.getDiskBufferCache(), harness.getDiskFileMapProvider(), fieldSerdes, numKeys,
+                harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(), harness.getOperationTracker(),
+                harness.getIOScheduler(), harness.getIOOperationCallbackProvider());
     }
 
     @Override
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeLifecycleTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeLifecycleTest.java
index a10185d..639fd47 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeLifecycleTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeLifecycleTest.java
@@ -16,7 +16,6 @@
 
 import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
 import edu.uci.ics.hyracks.api.io.FileReference;
-import edu.uci.ics.hyracks.api.io.IODeviceHandle;
 import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
 import edu.uci.ics.hyracks.storage.am.btree.OrderedIndexTestUtils;
 import edu.uci.ics.hyracks.storage.am.common.AbstractIndexLifecycleTest;
@@ -40,10 +39,8 @@
     @Override
     protected boolean persistentStateExists() throws Exception {
         // make sure all of the directories exist
-        for (IODeviceHandle handle : harness.getIOManager().getIODevices()) {
-            if (!new FileReference(handle, harness.getFileReference().getFile().getPath()).getFile().exists()) {
-                return false;
-            }
+        if (!new FileReference(harness.getFileReference().getFile()).getFile().exists()) {
+            return false;
         }
         return true;
     }
@@ -56,11 +53,10 @@
     @Override
     public void setup() throws Exception {
         harness.setUp();
-        testCtx = LSMBTreeTestContext.create(harness.getVirtualBufferCache(), harness.getIOManager(),
-                harness.getFileReference(), harness.getDiskBufferCache(), harness.getDiskFileMapProvider(),
-                fieldSerdes, fieldSerdes.length, harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
-                harness.getOperationTracker(), harness.getIOScheduler(), harness.getIOOperationCallbackProvider(),
-                harness.getIODeviceId());
+        testCtx = LSMBTreeTestContext.create(harness.getVirtualBufferCache(), harness.getFileReference(),
+                harness.getDiskBufferCache(), harness.getDiskFileMapProvider(), fieldSerdes, fieldSerdes.length,
+                harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(), harness.getOperationTracker(),
+                harness.getIOScheduler(), harness.getIOOperationCallbackProvider());
         index = testCtx.getIndex();
     }
 
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeMergeTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeMergeTest.java
index b84397c..e3c86df 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeMergeTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeMergeTest.java
@@ -50,11 +50,10 @@
     @Override
     protected OrderedIndexTestContext createTestContext(ISerializerDeserializer[] fieldSerdes, int numKeys,
             BTreeLeafFrameType leafType) throws Exception {
-        return LSMBTreeTestContext.create(harness.getVirtualBufferCache(), harness.getIOManager(),
-                harness.getFileReference(), harness.getDiskBufferCache(), harness.getDiskFileMapProvider(),
-                fieldSerdes, numKeys, harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
-                harness.getOperationTracker(), harness.getIOScheduler(), harness.getIOOperationCallbackProvider(),
-                harness.getIODeviceId());
+        return LSMBTreeTestContext.create(harness.getVirtualBufferCache(), harness.getFileReference(),
+                harness.getDiskBufferCache(), harness.getDiskFileMapProvider(), fieldSerdes, numKeys,
+                harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(), harness.getOperationTracker(),
+                harness.getIOScheduler(), harness.getIOOperationCallbackProvider());
     }
 
     @Override
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeMergeTestDriver.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeMergeTestDriver.java
index 97b5eb4..f7aa7f4 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeMergeTestDriver.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeMergeTestDriver.java
@@ -57,9 +57,15 @@
         for (int i = 0; i < maxTreesToMerge; i++) {
             for (int j = 0; j < i; j++) {
                 if (fieldSerdes[0] instanceof IntegerSerializerDeserializer) {
-                    orderedIndexTestUtils.bulkLoadIntTuples(ctx, numTuplesToInsert, getRandom());
+                    orderedIndexTestUtils.insertIntTuples(ctx, numTuplesToInsert, getRandom());
+                    // Deactivate and the re-activate the index to force it flush its in memory component
+                    ctx.getIndex().deactivate();
+                    ctx.getIndex().activate();
                 } else if (fieldSerdes[0] instanceof UTF8StringSerializerDeserializer) {
-                    orderedIndexTestUtils.bulkLoadStringTuples(ctx, numTuplesToInsert, getRandom());
+                    orderedIndexTestUtils.insertStringTuples(ctx, numTuplesToInsert, getRandom());
+                    // Deactivate and the re-activate the index to force it flush its in memory component
+                    ctx.getIndex().deactivate();
+                    ctx.getIndex().activate();
                 }
             }
 
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeModificationOperationCallbackTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeModificationOperationCallbackTest.java
index 0c5bf76..77b65ca 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeModificationOperationCallbackTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeModificationOperationCallbackTest.java
@@ -42,13 +42,13 @@
 
     @Override
     protected void createIndexInstance() throws Exception {
-        index = LSMBTreeUtils.createLSMTree(harness.getVirtualBufferCache(), harness.getIOManager(),
-                harness.getFileReference(), harness.getDiskBufferCache(), harness.getDiskFileMapProvider(),
+        index = LSMBTreeUtils.createLSMTree(harness.getVirtualBufferCache(), harness.getFileReference(),
+                harness.getDiskBufferCache(), harness.getDiskFileMapProvider(),
                 SerdeUtils.serdesToTypeTraits(keySerdes),
                 SerdeUtils.serdesToComparatorFactories(keySerdes, keySerdes.length), bloomFilterKeyFields,
                 harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
                 NoOpOperationTrackerProvider.INSTANCE.getOperationTracker(null), harness.getIOScheduler(),
-                harness.getIOOperationCallbackProvider(), harness.getIODeviceId());
+                harness.getIOOperationCallbackProvider());
     }
 
     @Override
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeMultiBulkLoadTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeMultiBulkLoadTest.java
index f0b230a..a7c8b81 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeMultiBulkLoadTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeMultiBulkLoadTest.java
@@ -52,15 +52,14 @@
     @Override
     protected OrderedIndexTestContext createTestContext(ISerializerDeserializer[] fieldSerdes, int numKeys,
             BTreeLeafFrameType leafType) throws Exception {
-        return LSMBTreeTestContext.create(harness.getVirtualBufferCache(), harness.getIOManager(),
-                harness.getFileReference(), harness.getDiskBufferCache(), harness.getDiskFileMapProvider(),
-                fieldSerdes, numKeys, harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
-                harness.getOperationTracker(), harness.getIOScheduler(), harness.getIOOperationCallbackProvider(),
-                harness.getIODeviceId());
+        return LSMBTreeTestContext.create(harness.getVirtualBufferCache(), harness.getFileReference(),
+                harness.getDiskBufferCache(), harness.getDiskFileMapProvider(), fieldSerdes, numKeys,
+                harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(), harness.getOperationTracker(),
+                harness.getIOScheduler(), harness.getIOOperationCallbackProvider());
     }
 
     @Override
     protected Random getRandom() {
         return harness.getRandom();
     }
-}
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeSearchOperationCallbackTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeSearchOperationCallbackTest.java
index e0db517..e15e80a 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeSearchOperationCallbackTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeSearchOperationCallbackTest.java
@@ -49,13 +49,13 @@
 
     @Override
     protected void createIndexInstance() throws Exception {
-        index = LSMBTreeUtils.createLSMTree(harness.getVirtualBufferCache(), harness.getIOManager(),
-                harness.getFileReference(), harness.getDiskBufferCache(), harness.getDiskFileMapProvider(),
+        index = LSMBTreeUtils.createLSMTree(harness.getVirtualBufferCache(), harness.getFileReference(),
+                harness.getDiskBufferCache(), harness.getDiskFileMapProvider(),
                 SerdeUtils.serdesToTypeTraits(keySerdes),
                 SerdeUtils.serdesToComparatorFactories(keySerdes, keySerdes.length), bloomFilterKeyFields,
                 harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
                 NoOpOperationTrackerProvider.INSTANCE.getOperationTracker(null), harness.getIOScheduler(),
-                harness.getIOOperationCallbackProvider(), harness.getIODeviceId());
+                harness.getIOOperationCallbackProvider());
     }
 
     @Override
@@ -94,7 +94,7 @@
         private int expectedAfterBlock;
         private int expectedTupleToBeLockedValue;
 
-        public SearchTask() {
+        public SearchTask() throws HyracksDataException {
             this.cb = new SynchronizingSearchOperationCallback();
             this.accessor = index.createAccessor(NoOpOperationCallback.INSTANCE, cb);
             this.cursor = accessor.createSearchCursor();
@@ -210,7 +210,7 @@
         private final ArrayTupleBuilder builder;
         private final ArrayTupleReference tuple;
 
-        public InsertionTask() {
+        public InsertionTask() throws HyracksDataException {
             this.accessor = index.createAccessor(NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE);
             this.builder = new ArrayTupleBuilder(NUM_KEY_FIELDS);
             this.tuple = new ArrayTupleReference();
@@ -222,11 +222,15 @@
             try {
                 insertTaskStarted = true;
 
-                // bulkload [101, 150] & [151, 200] as two separate disk components 
+                // bulkload [101, 150] and then insert [151, 200] and make sure it reaches disk, thus we will have two separate disk components 
                 // insert [50, 100] & [301, 350] to the in-memory component
                 // delete tuple 151
                 bulkloadIntTupleRange(101, 150);
-                bulkloadIntTupleRange(151, 200);
+                insertIntTupleRange(151, 200);
+                // Deactivate and the re-activate the index to force it flush its in memory component
+                index.deactivate();
+                index.activate();
+
                 insertIntTupleRange(50, 100);
                 insertIntTupleRange(301, 350);
                 int tupleTobeDeletedValue = 151;
@@ -270,7 +274,7 @@
                 throw new IllegalArgumentException("Invalid range: [" + begin + ", " + end + "]");
             }
 
-            IIndexBulkLoader bulkloader = index.createBulkLoader(1.0f, false, end - begin);
+            IIndexBulkLoader bulkloader = index.createBulkLoader(1.0f, false, end - begin, true);
             for (int i = begin; i <= end; i++) {
                 TupleUtils.createIntegerTuple(builder, tuple, i);
                 bulkloader.add(tuple);
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeUpdateTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeUpdateTest.java
index 59b837c..db221a2 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeUpdateTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeUpdateTest.java
@@ -51,11 +51,10 @@
     @Override
     protected OrderedIndexTestContext createTestContext(ISerializerDeserializer[] fieldSerdes, int numKeys,
             BTreeLeafFrameType leafType) throws Exception {
-        return LSMBTreeTestContext.create(harness.getVirtualBufferCache(), harness.getIOManager(),
-                harness.getFileReference(), harness.getDiskBufferCache(), harness.getDiskFileMapProvider(),
-                fieldSerdes, numKeys, harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
-                harness.getOperationTracker(), harness.getIOScheduler(), harness.getIOOperationCallbackProvider(),
-                harness.getIODeviceId());
+        return LSMBTreeTestContext.create(harness.getVirtualBufferCache(), harness.getFileReference(),
+                harness.getDiskBufferCache(), harness.getDiskFileMapProvider(), fieldSerdes, numKeys,
+                harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(), harness.getOperationTracker(),
+                harness.getIOScheduler(), harness.getIOOperationCallbackProvider());
     }
 
     @Override
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/multithread/LSMBTreeMultiThreadTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/multithread/LSMBTreeMultiThreadTest.java
index c5db1c1..9dfa713 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/multithread/LSMBTreeMultiThreadTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/multithread/LSMBTreeMultiThreadTest.java
@@ -50,11 +50,10 @@
     @Override
     protected ITreeIndex createIndex(ITypeTraits[] typeTraits, IBinaryComparatorFactory[] cmpFactories,
             int[] bloomFilterKeyFields) throws TreeIndexException {
-        return LSMBTreeUtils.createLSMTree(harness.getVirtualBufferCache(), harness.getIOManager(),
-                harness.getFileReference(), harness.getDiskBufferCache(), harness.getDiskFileMapProvider(), typeTraits,
-                cmpFactories, bloomFilterKeyFields, harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
-                harness.getOperationTracker(), harness.getIOScheduler(), harness.getIOOperationCallbackProvider(),
-                harness.getIODeviceId());
+        return LSMBTreeUtils.createLSMTree(harness.getVirtualBufferCache(), harness.getFileReference(),
+                harness.getDiskBufferCache(), harness.getDiskFileMapProvider(), typeTraits, cmpFactories,
+                bloomFilterKeyFields, harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
+                harness.getOperationTracker(), harness.getIOScheduler(), harness.getIOOperationCallbackProvider());
     }
 
     @Override
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/multithread/LSMBTreeTestWorker.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/multithread/LSMBTreeTestWorker.java
index 80165d7..8a37b4e 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/multithread/LSMBTreeTestWorker.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/multithread/LSMBTreeTestWorker.java
@@ -41,7 +41,8 @@
     private final ArrayTupleBuilder deleteTb;
     private final ArrayTupleReference deleteTuple = new ArrayTupleReference();
 
-    public LSMBTreeTestWorker(DataGenThread dataGen, TestOperationSelector opSelector, IIndex index, int numBatches) {
+    public LSMBTreeTestWorker(DataGenThread dataGen, TestOperationSelector opSelector, IIndex index, int numBatches)
+            throws HyracksDataException {
         super(dataGen, opSelector, index, numBatches);
         lsmBTree = (LSMBTree) index;
         numKeyFields = lsmBTree.getComparatorFactories().length;
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/multithread/LSMBTreeTestWorkerFactory.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/multithread/LSMBTreeTestWorkerFactory.java
index 6296fed..2015a66 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/multithread/LSMBTreeTestWorkerFactory.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/multithread/LSMBTreeTestWorkerFactory.java
@@ -15,6 +15,7 @@
 
 package edu.uci.ics.hyracks.storage.am.lsm.btree.multithread;
 
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.storage.am.common.AbstractIndexTestWorker;
 import edu.uci.ics.hyracks.storage.am.common.IIndexTestWorkerFactory;
 import edu.uci.ics.hyracks.storage.am.common.TestOperationSelector;
@@ -23,8 +24,8 @@
 
 public class LSMBTreeTestWorkerFactory implements IIndexTestWorkerFactory {
     @Override
-    public AbstractIndexTestWorker create(DataGenThread dataGen, TestOperationSelector opSelector,
-            IIndex index, int numBatches) {
+    public AbstractIndexTestWorker create(DataGenThread dataGen, TestOperationSelector opSelector, IIndex index,
+            int numBatches) throws HyracksDataException {
         return new LSMBTreeTestWorker(dataGen, opSelector, index, numBatches);
     }
 }
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/perf/BTreeBulkLoadRunner.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/perf/BTreeBulkLoadRunner.java
index d364955..52cbddb 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/perf/BTreeBulkLoadRunner.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/perf/BTreeBulkLoadRunner.java
@@ -37,7 +37,7 @@
     public long runExperiment(DataGenThread dataGen, int numThreads) throws Exception {
         btree.create();
         long start = System.currentTimeMillis();
-        IIndexBulkLoader bulkLoader = btree.createBulkLoader(1.0f, false, 0L);
+        IIndexBulkLoader bulkLoader = btree.createBulkLoader(1.0f, false, 0L, true);
         for (int i = 0; i < numBatches; i++) {
             TupleBatch batch = dataGen.tupleBatchQueue.take();
             for (int j = 0; j < batch.size(); j++) {
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/perf/LSMTreeRunner.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/perf/LSMTreeRunner.java
index 1307212..fc6642c 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/perf/LSMTreeRunner.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/perf/LSMTreeRunner.java
@@ -91,9 +91,9 @@
         IVirtualBufferCache virtualBufferCache = new VirtualBufferCache(new HeapBufferAllocator(), inMemPageSize,
                 inMemNumPages);
         this.ioScheduler = SynchronousScheduler.INSTANCE;
-        lsmtree = LSMBTreeUtils.createLSMTree(virtualBufferCache, ioManager, file, bufferCache, fmp, typeTraits,
-                cmpFactories, bloomFilterKeyFields, bloomFilterFalsePositiveRate, NoMergePolicy.INSTANCE,
-                new ThreadCountingTracker(), ioScheduler, NoOpIOOperationCallback.INSTANCE, ioDeviceId);
+        lsmtree = LSMBTreeUtils.createLSMTree(virtualBufferCache, file, bufferCache, fmp, typeTraits, cmpFactories,
+                bloomFilterKeyFields, bloomFilterFalsePositiveRate, NoMergePolicy.INSTANCE,
+                new ThreadCountingTracker(), ioScheduler, NoOpIOOperationCallback.INSTANCE);
     }
 
     @Override
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/util/LSMBTreeTestContext.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/util/LSMBTreeTestContext.java
index 5f8809b..ab78ee61 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/util/LSMBTreeTestContext.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/util/LSMBTreeTestContext.java
@@ -20,8 +20,8 @@
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
 import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
 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.control.nc.io.IOManager;
 import edu.uci.ics.hyracks.dataflow.common.util.SerdeUtils;
 import edu.uci.ics.hyracks.storage.am.btree.OrderedIndexTestContext;
 import edu.uci.ics.hyracks.storage.am.common.CheckTuple;
@@ -38,7 +38,7 @@
 @SuppressWarnings("rawtypes")
 public final class LSMBTreeTestContext extends OrderedIndexTestContext {
 
-    public LSMBTreeTestContext(ISerializerDeserializer[] fieldSerdes, ITreeIndex treeIndex) {
+    public LSMBTreeTestContext(ISerializerDeserializer[] fieldSerdes, ITreeIndex treeIndex) throws HyracksDataException {
         super(fieldSerdes, treeIndex);
     }
 
@@ -62,20 +62,20 @@
         upsertCheckTuple(checkTuple, checkTuples);
     }
 
-    public static LSMBTreeTestContext create(IVirtualBufferCache virtualBufferCache, IOManager ioManager,
-            FileReference file, IBufferCache diskBufferCache, IFileMapProvider diskFileMapProvider,
-            ISerializerDeserializer[] fieldSerdes, int numKeyFields, double bloomFilterFalsePositiveRate,
-            ILSMMergePolicy mergePolicy, ILSMOperationTracker opTracker, ILSMIOOperationScheduler ioScheduler,
-            ILSMIOOperationCallbackProvider ioOpCallbackProvider, int ioDeviceId) throws Exception {
+    public static LSMBTreeTestContext create(IVirtualBufferCache virtualBufferCache, FileReference file,
+            IBufferCache diskBufferCache, IFileMapProvider diskFileMapProvider, ISerializerDeserializer[] fieldSerdes,
+            int numKeyFields, double bloomFilterFalsePositiveRate, ILSMMergePolicy mergePolicy,
+            ILSMOperationTracker opTracker, ILSMIOOperationScheduler ioScheduler,
+            ILSMIOOperationCallbackProvider ioOpCallbackProvider) throws Exception {
         ITypeTraits[] typeTraits = SerdeUtils.serdesToTypeTraits(fieldSerdes);
         IBinaryComparatorFactory[] cmpFactories = SerdeUtils.serdesToComparatorFactories(fieldSerdes, numKeyFields);
         int[] bloomFilterKeyFields = new int[numKeyFields];
         for (int i = 0; i < numKeyFields; ++i) {
             bloomFilterKeyFields[i] = i;
         }
-        LSMBTree lsmTree = LSMBTreeUtils.createLSMTree(virtualBufferCache, ioManager, file, diskBufferCache,
-                diskFileMapProvider, typeTraits, cmpFactories, bloomFilterKeyFields, bloomFilterFalsePositiveRate,
-                mergePolicy, opTracker, ioScheduler, ioOpCallbackProvider, ioDeviceId);
+        LSMBTree lsmTree = LSMBTreeUtils.createLSMTree(virtualBufferCache, file, diskBufferCache, diskFileMapProvider,
+                typeTraits, cmpFactories, bloomFilterKeyFields, bloomFilterFalsePositiveRate, mergePolicy, opTracker,
+                ioScheduler, ioOpCallbackProvider);
         LSMBTreeTestContext testCtx = new LSMBTreeTestContext(fieldSerdes, lsmTree);
         return testCtx;
     }
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/util/LSMBTreeTestHarness.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/util/LSMBTreeTestHarness.java
index 94aa841..e643cf9 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/util/LSMBTreeTestHarness.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/util/LSMBTreeTestHarness.java
@@ -107,36 +107,36 @@
     }
 
     public void setUp() throws HyracksException {
-        onDiskDir = "lsm_btree_" + simpleDateFormat.format(new Date()) + sep;
+        ioManager = TestStorageManagerComponentHolder.getIOManager();
+        ioDeviceId = 0;
+        onDiskDir = ioManager.getIODevices().get(ioDeviceId).getPath() + sep + "lsm_btree_"
+                + simpleDateFormat.format(new Date()) + sep;
         file = new FileReference(new File(onDiskDir));
         ctx = TestUtils.create(getHyracksFrameSize());
         TestStorageManagerComponentHolder.init(diskPageSize, diskNumPages, diskMaxOpenFiles);
         diskBufferCache = TestStorageManagerComponentHolder.getBufferCache(ctx);
         diskFileMapProvider = TestStorageManagerComponentHolder.getFileMapProvider(ctx);
         virtualBufferCache = new VirtualBufferCache(new HeapBufferAllocator(), memPageSize, memNumPages);
-        ioManager = TestStorageManagerComponentHolder.getIOManager();
-        ioDeviceId = 0;
         rnd.setSeed(RANDOM_SEED);
     }
 
     public void tearDown() throws HyracksDataException {
         diskBufferCache.close();
-        for (IODeviceHandle dev : ioManager.getIODevices()) {
-            File dir = new File(dev.getPath(), onDiskDir);
-            FilenameFilter filter = new FilenameFilter() {
-                public boolean accept(File dir, String name) {
-                    return !name.startsWith(".");
-                }
-            };
-            String[] files = dir.list(filter);
-            if (files != null) {
-                for (String fileName : files) {
-                    File file = new File(dir.getPath() + File.separator + fileName);
-                    file.delete();
-                }
+        IODeviceHandle dev = ioManager.getIODevices().get(ioDeviceId);
+        File dir = new File(dev.getPath(), onDiskDir);
+        FilenameFilter filter = new FilenameFilter() {
+            public boolean accept(File dir, String name) {
+                return !name.startsWith(".");
             }
-            dir.delete();
+        };
+        String[] files = dir.list(filter);
+        if (files != null) {
+            for (String fileName : files) {
+                File file = new File(dir.getPath() + File.separator + fileName);
+                file.delete();
+            }
         }
+        dir.delete();
     }
 
     public int getDiskPageSize() {
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-common-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/common/DummyLSMIndexFileManager.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-common-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/common/DummyLSMIndexFileManager.java
index 19bf21f..e4dd974 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-common-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/common/DummyLSMIndexFileManager.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-common-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/common/DummyLSMIndexFileManager.java
@@ -21,8 +21,6 @@
 
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.api.io.FileReference;
-import edu.uci.ics.hyracks.api.io.IIOManager;
-import edu.uci.ics.hyracks.api.io.IODeviceHandle;
 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;
@@ -31,15 +29,15 @@
 
 public class DummyLSMIndexFileManager extends AbstractLSMIndexFileManager {
 
-    public DummyLSMIndexFileManager(IIOManager ioManager, IFileMapProvider fileMapProvider, FileReference file,
-            TreeIndexFactory<? extends ITreeIndex> treeFactory, int ioDeviceId) {
-        super(ioManager, fileMapProvider, file, treeFactory, ioDeviceId);
+    public DummyLSMIndexFileManager(IFileMapProvider fileMapProvider, FileReference file,
+            TreeIndexFactory<? extends ITreeIndex> treeFactory) {
+        super(fileMapProvider, file, treeFactory);
     }
 
-    protected void cleanupAndGetValidFilesInternal(IODeviceHandle dev, FilenameFilter filter,
+    protected void cleanupAndGetValidFilesInternal(FilenameFilter filter,
             TreeIndexFactory<? extends ITreeIndex> treeFactory, ArrayList<ComparableFileName> allFiles)
             throws HyracksDataException, IndexException {
-        File dir = new File(dev.getPath(), baseDir);
+        File dir = new File(baseDir);
         String[] files = dir.list(filter);
         for (String fileName : files) {
             File file = new File(dir.getPath() + File.separator + fileName);
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-common-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/common/LSMIndexFileManagerTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-common-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/common/LSMIndexFileManagerTest.java
index 5695eb8..d9fdeb3 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-common-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/common/LSMIndexFileManagerTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-common-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/common/LSMIndexFileManagerTest.java
@@ -61,7 +61,8 @@
         TestStorageManagerComponentHolder.init(DEFAULT_PAGE_SIZE, DEFAULT_NUM_PAGES, DEFAULT_MAX_OPEN_FILES);
         ioManager = TestStorageManagerComponentHolder.getIOManager();
         fileMapProvider = TestStorageManagerComponentHolder.getFileMapProvider(null);
-        baseDir = "lsm_tree" + simpleDateFormat.format(new Date()) + sep;
+        baseDir = ioManager.getIODevices().get(DEFAULT_IO_DEVICE_ID).getPath() + sep + "lsm_tree"
+                + simpleDateFormat.format(new Date()) + sep;
         File f = new File(baseDir);
         f.mkdirs();
         file = new FileReference(f);
@@ -74,8 +75,7 @@
     }
 
     public void sortOrderTest(boolean testFlushFileName) throws InterruptedException, HyracksDataException {
-        ILSMIndexFileManager fileManager = new DummyLSMIndexFileManager(ioManager, fileMapProvider, file,
-                new DummyTreeFactory(), DEFAULT_IO_DEVICE_ID);
+        ILSMIndexFileManager fileManager = new DummyLSMIndexFileManager(fileMapProvider, file, new DummyTreeFactory());
         LinkedList<String> fileNames = new LinkedList<String>();
 
         int numFileNames = 100;
@@ -115,8 +115,7 @@
     }
 
     public void cleanInvalidFilesTest(IOManager ioManager) throws InterruptedException, IOException, IndexException {
-        ILSMIndexFileManager fileManager = new DummyLSMIndexFileManager(ioManager, fileMapProvider, file,
-                new DummyTreeFactory(), DEFAULT_IO_DEVICE_ID);
+        ILSMIndexFileManager fileManager = new DummyLSMIndexFileManager(fileMapProvider, file, new DummyTreeFactory());
         fileManager.createDirs();
 
         List<FileReference> flushFiles = new ArrayList<FileReference>();
@@ -190,7 +189,7 @@
 
         // Make sure invalid files were removed from the IODevices.
         ArrayList<String> remainingFiles = new ArrayList<String>();
-        File dir = new File(ioManager.getIODevices().get(DEFAULT_IO_DEVICE_ID).getPath(), baseDir);
+        File dir = new File(baseDir);
         FilenameFilter filter = new FilenameFilter() {
             public boolean accept(File dir, String name) {
                 return !name.startsWith(".");
@@ -218,7 +217,7 @@
     }
 
     private void cleanDirs(IOManager ioManager) {
-        File dir = new File(ioManager.getIODevices().get(DEFAULT_IO_DEVICE_ID).getPath(), baseDir);
+        File dir = new File(baseDir);
         FilenameFilter filter = new FilenameFilter() {
             public boolean accept(File dir, String name) {
                 return !name.startsWith(".");
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/LSMInvertedIndexBulkLoadTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/LSMInvertedIndexBulkLoadTest.java
index e254fba..ee09f2f 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/LSMInvertedIndexBulkLoadTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/LSMInvertedIndexBulkLoadTest.java
@@ -21,6 +21,6 @@
 public class LSMInvertedIndexBulkLoadTest extends AbstractInvertedIndexLoadTest {
 
     public LSMInvertedIndexBulkLoadTest() {
-        super(InvertedIndexType.LSM, true, 1);
+        super(InvertedIndexType.LSM, true);
     }
 }
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/LSMInvertedIndexInsertTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/LSMInvertedIndexInsertTest.java
index 07afebe..748dd32 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/LSMInvertedIndexInsertTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/LSMInvertedIndexInsertTest.java
@@ -21,6 +21,6 @@
 public class LSMInvertedIndexInsertTest extends AbstractInvertedIndexLoadTest {
 
     public LSMInvertedIndexInsertTest() {
-        super(InvertedIndexType.LSM, false, 1);
+        super(InvertedIndexType.LSM, false);
     }
 }
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/LSMInvertedIndexMergeTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/LSMInvertedIndexMergeTest.java
index 8c0c597..5115b7e 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/LSMInvertedIndexMergeTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/LSMInvertedIndexMergeTest.java
@@ -34,7 +34,7 @@
     private final int maxTreesToMerge = AccessMethodTestsConfig.LSM_INVINDEX_MAX_TREES_TO_MERGE;
 
     public LSMInvertedIndexMergeTest() {
-        super(InvertedIndexType.LSM, true, 1);
+        super(InvertedIndexType.LSM, false);
     }
 
     @Override
@@ -48,11 +48,10 @@
 
         for (int i = 0; i < maxTreesToMerge; i++) {
             for (int j = 0; j < i; j++) {
-                if (bulkLoad) {
-                    LSMInvertedIndexTestUtils.bulkLoadInvIndex(testCtx, tupleGen, NUM_DOCS_TO_INSERT);
-                } else {
-                    LSMInvertedIndexTestUtils.insertIntoInvIndex(testCtx, tupleGen, NUM_DOCS_TO_INSERT);
-                }
+                LSMInvertedIndexTestUtils.insertIntoInvIndex(testCtx, tupleGen, NUM_DOCS_TO_INSERT);
+                // Deactivate and the re-activate the index to force it flush its in memory component
+                invIndex.deactivate();
+                invIndex.activate();
             }
             // Perform merge.
             invIndexAccessor.scheduleMerge(NoOpIOOperationCallback.INSTANCE);
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/LSMInvertedIndexMultiBulkLoadTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/LSMInvertedIndexMultiBulkLoadTest.java
deleted file mode 100644
index 0d8f59c..0000000
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/LSMInvertedIndexMultiBulkLoadTest.java
+++ /dev/null
@@ -1,27 +0,0 @@
-/*
- * Copyright 2009-2013 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package edu.uci.ics.hyracks.storage.am.lsm.invertedindex;
-
-import edu.uci.ics.hyracks.storage.am.config.AccessMethodTestsConfig;
-import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.common.AbstractInvertedIndexLoadTest;
-import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.util.LSMInvertedIndexTestContext.InvertedIndexType;
-
-public class LSMInvertedIndexMultiBulkLoadTest extends AbstractInvertedIndexLoadTest {
-
-    public LSMInvertedIndexMultiBulkLoadTest() {
-        super(InvertedIndexType.LSM, true, AccessMethodTestsConfig.LSM_INVINDEX_NUM_BULKLOAD_ROUNDS);
-    }
-}
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/PartitionedLSMInvertedIndexBulkLoadTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/PartitionedLSMInvertedIndexBulkLoadTest.java
index 6c2da74..0f0345f 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/PartitionedLSMInvertedIndexBulkLoadTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/PartitionedLSMInvertedIndexBulkLoadTest.java
@@ -21,6 +21,6 @@
 public class PartitionedLSMInvertedIndexBulkLoadTest extends AbstractInvertedIndexLoadTest {
 
     public PartitionedLSMInvertedIndexBulkLoadTest() {
-        super(InvertedIndexType.PARTITIONED_LSM, true, 1);
+        super(InvertedIndexType.PARTITIONED_LSM, true);
     }
 }
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/PartitionedLSMInvertedIndexInsertTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/PartitionedLSMInvertedIndexInsertTest.java
index 7a93069..608f2fc 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/PartitionedLSMInvertedIndexInsertTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/PartitionedLSMInvertedIndexInsertTest.java
@@ -21,6 +21,6 @@
 public class PartitionedLSMInvertedIndexInsertTest extends AbstractInvertedIndexLoadTest {
 
     public PartitionedLSMInvertedIndexInsertTest() {
-        super(InvertedIndexType.PARTITIONED_LSM, false, 1);
+        super(InvertedIndexType.PARTITIONED_LSM, false);
     }
 }
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/PartitionedLSMInvertedIndexMergeTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/PartitionedLSMInvertedIndexMergeTest.java
index bccf27b..523557d 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/PartitionedLSMInvertedIndexMergeTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/PartitionedLSMInvertedIndexMergeTest.java
@@ -34,7 +34,7 @@
     private final int maxTreesToMerge = AccessMethodTestsConfig.LSM_INVINDEX_MAX_TREES_TO_MERGE;
 
     public PartitionedLSMInvertedIndexMergeTest() {
-        super(InvertedIndexType.PARTITIONED_LSM, true, 1);
+        super(InvertedIndexType.PARTITIONED_LSM, false);
     }
 
     @Override
@@ -48,11 +48,10 @@
 
         for (int i = 0; i < maxTreesToMerge; i++) {
             for (int j = 0; j < i; j++) {
-                if (bulkLoad) {
-                    LSMInvertedIndexTestUtils.bulkLoadInvIndex(testCtx, tupleGen, NUM_DOCS_TO_INSERT);
-                } else {
-                    LSMInvertedIndexTestUtils.insertIntoInvIndex(testCtx, tupleGen, NUM_DOCS_TO_INSERT);
-                }
+                LSMInvertedIndexTestUtils.insertIntoInvIndex(testCtx, tupleGen, NUM_DOCS_TO_INSERT);
+                // Deactivate and the re-activate the index to force it flush its in memory component
+                invIndex.deactivate();
+                invIndex.activate();
             }
             // Perform merge.
             invIndexAccessor.scheduleMerge(NoOpIOOperationCallback.INSTANCE);
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/PartitionedLSMInvertedIndexMultiBulkLoadTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/PartitionedLSMInvertedIndexMultiBulkLoadTest.java
deleted file mode 100644
index 71194f5..0000000
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/PartitionedLSMInvertedIndexMultiBulkLoadTest.java
+++ /dev/null
@@ -1,27 +0,0 @@
-/*
- * Copyright 2009-2013 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package edu.uci.ics.hyracks.storage.am.lsm.invertedindex;
-
-import edu.uci.ics.hyracks.storage.am.config.AccessMethodTestsConfig;
-import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.common.AbstractInvertedIndexLoadTest;
-import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.util.LSMInvertedIndexTestContext.InvertedIndexType;
-
-public class PartitionedLSMInvertedIndexMultiBulkLoadTest extends AbstractInvertedIndexLoadTest {
-
-    public PartitionedLSMInvertedIndexMultiBulkLoadTest() {
-        super(InvertedIndexType.PARTITIONED_LSM, true, AccessMethodTestsConfig.LSM_INVINDEX_NUM_BULKLOAD_ROUNDS);
-    }
-}
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/common/AbstractInvertedIndexLoadTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/common/AbstractInvertedIndexLoadTest.java
index c3d6a16..8ccc203 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/common/AbstractInvertedIndexLoadTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/common/AbstractInvertedIndexLoadTest.java
@@ -29,12 +29,10 @@
 public abstract class AbstractInvertedIndexLoadTest extends AbstractInvertedIndexTest {
 
     protected final boolean bulkLoad;
-    protected final int numRounds;
 
-    public AbstractInvertedIndexLoadTest(InvertedIndexType invIndexType, boolean bulkLoad, int numRounds) {
+    public AbstractInvertedIndexLoadTest(InvertedIndexType invIndexType, boolean bulkLoad) {
         super(invIndexType);
         this.bulkLoad = bulkLoad;
-        this.numRounds = numRounds;
     }
 
     protected void runTest(LSMInvertedIndexTestContext testCtx, TupleGenerator tupleGen) throws IOException,
@@ -43,15 +41,13 @@
         invIndex.create();
         invIndex.activate();
 
-        for (int i = 0; i < numRounds; i++) {
-            if (bulkLoad) {
-                LSMInvertedIndexTestUtils.bulkLoadInvIndex(testCtx, tupleGen, NUM_DOCS_TO_INSERT);
-            } else {
-                LSMInvertedIndexTestUtils.insertIntoInvIndex(testCtx, tupleGen, NUM_DOCS_TO_INSERT);
-            }
-            validateAndCheckIndex(testCtx);
-            runTinySearchWorkload(testCtx, tupleGen);
+        if (bulkLoad) {
+            LSMInvertedIndexTestUtils.bulkLoadInvIndex(testCtx, tupleGen, NUM_DOCS_TO_INSERT);
+        } else {
+            LSMInvertedIndexTestUtils.insertIntoInvIndex(testCtx, tupleGen, NUM_DOCS_TO_INSERT);
         }
+        validateAndCheckIndex(testCtx);
+        runTinySearchWorkload(testCtx, tupleGen);
 
         invIndex.deactivate();
         invIndex.destroy();
@@ -59,7 +55,8 @@
 
     @Test
     public void wordTokensInvIndexTest() throws IOException, IndexException {
-        LSMInvertedIndexTestContext testCtx = LSMInvertedIndexTestUtils.createWordInvIndexTestContext(harness, invIndexType);
+        LSMInvertedIndexTestContext testCtx = LSMInvertedIndexTestUtils.createWordInvIndexTestContext(harness,
+                invIndexType);
         TupleGenerator tupleGen = LSMInvertedIndexTestUtils.createStringDocumentTupleGen(harness.getRandom());
         runTest(testCtx, tupleGen);
     }
@@ -74,7 +71,8 @@
 
     @Test
     public void ngramTokensInvIndexTest() throws IOException, IndexException {
-        LSMInvertedIndexTestContext testCtx = LSMInvertedIndexTestUtils.createNGramInvIndexTestContext(harness, invIndexType);
+        LSMInvertedIndexTestContext testCtx = LSMInvertedIndexTestUtils.createNGramInvIndexTestContext(harness,
+                invIndexType);
         TupleGenerator tupleGen = LSMInvertedIndexTestUtils.createPersonNamesTupleGen(harness.getRandom());
         runTest(testCtx, tupleGen);
     }
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/common/LSMInvertedIndexTestHarness.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/common/LSMInvertedIndexTestHarness.java
index 7407c24..7276464 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/common/LSMInvertedIndexTestHarness.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/common/LSMInvertedIndexTestHarness.java
@@ -108,7 +108,10 @@
     }
 
     public void setUp() throws HyracksException {
-        onDiskDir = "lsm_invertedindex_" + simpleDateFormat.format(new Date()) + sep;
+        ioManager = TestStorageManagerComponentHolder.getIOManager();
+        ioDeviceId = 0;
+        onDiskDir = ioManager.getIODevices().get(ioDeviceId).getPath() + sep + "lsm_invertedindex_"
+                + simpleDateFormat.format(new Date()) + sep;
         ctx = TestUtils.create(getHyracksFrameSize());
         TestStorageManagerComponentHolder.init(diskPageSize, diskNumPages, diskMaxOpenFiles);
         diskBufferCache = TestStorageManagerComponentHolder.getBufferCache(ctx);
@@ -117,30 +120,27 @@
                 memPageSize, memNumPages));
         virtualBufferCache.open();
         virtualFreePageManager = new VirtualFreePageManager(memNumPages);
-        ioManager = TestStorageManagerComponentHolder.getIOManager();
-        ioDeviceId = 0;
         rnd.setSeed(RANDOM_SEED);
         invIndexFileRef = ioManager.getIODevices().get(0).createFileReference(onDiskDir + invIndexFileName);
     }
 
     public void tearDown() throws HyracksDataException {
         diskBufferCache.close();
-        for (IODeviceHandle dev : ioManager.getIODevices()) {
-            File dir = new File(dev.getPath(), onDiskDir);
-            FilenameFilter filter = new FilenameFilter() {
-                public boolean accept(File dir, String name) {
-                    return !name.startsWith(".");
-                }
-            };
-            String[] files = dir.list(filter);
-            if (files != null) {
-                for (String fileName : files) {
-                    File file = new File(dir.getPath() + File.separator + fileName);
-                    file.delete();
-                }
+        IODeviceHandle dev = ioManager.getIODevices().get(ioDeviceId);
+        File dir = new File(dev.getPath(), onDiskDir);
+        FilenameFilter filter = new FilenameFilter() {
+            public boolean accept(File dir, String name) {
+                return !name.startsWith(".");
             }
-            dir.delete();
+        };
+        String[] files = dir.list(filter);
+        if (files != null) {
+            for (String fileName : files) {
+                File file = new File(dir.getPath() + File.separator + fileName);
+                file.delete();
+            }
         }
+        dir.delete();
         virtualBufferCache.close();
     }
 
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/inmemory/InMemoryInvertedIndexInsertTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/inmemory/InMemoryInvertedIndexInsertTest.java
index bba0f36..8bc2817 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/inmemory/InMemoryInvertedIndexInsertTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/inmemory/InMemoryInvertedIndexInsertTest.java
@@ -21,6 +21,6 @@
 public class InMemoryInvertedIndexInsertTest extends AbstractInvertedIndexLoadTest {
     
     public InMemoryInvertedIndexInsertTest() {
-        super(InvertedIndexType.INMEMORY, false, 1);
+        super(InvertedIndexType.INMEMORY, false);
     }
 }
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/inmemory/PartitionedInMemoryInvertedIndexInsertTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/inmemory/PartitionedInMemoryInvertedIndexInsertTest.java
index fa8eee0..46ea72c 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/inmemory/PartitionedInMemoryInvertedIndexInsertTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/inmemory/PartitionedInMemoryInvertedIndexInsertTest.java
@@ -21,6 +21,6 @@
 public class PartitionedInMemoryInvertedIndexInsertTest extends AbstractInvertedIndexLoadTest {
 
     public PartitionedInMemoryInvertedIndexInsertTest() {
-        super(InvertedIndexType.PARTITIONED_INMEMORY, false, 1);
+        super(InvertedIndexType.PARTITIONED_INMEMORY, false);
     }
 }
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/multithread/LSMInvertedIndexTestWorker.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/multithread/LSMInvertedIndexTestWorker.java
index b4875a8..e1570af 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/multithread/LSMInvertedIndexTestWorker.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/multithread/LSMInvertedIndexTestWorker.java
@@ -50,7 +50,7 @@
             new ConjunctiveSearchModifier(), new JaccardSearchModifier(0.8f), new JaccardSearchModifier(0.5f) };
 
     public LSMInvertedIndexTestWorker(DataGenThread dataGen, TestOperationSelector opSelector, IIndex index,
-            int numBatches) {
+            int numBatches) throws HyracksDataException {
         super(dataGen, opSelector, index, numBatches);
         invIndex = (LSMInvertedIndex) index;
     }
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/multithread/LSMInvertedIndexWorkerFactory.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/multithread/LSMInvertedIndexWorkerFactory.java
index 6964f3d..c211edb 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/multithread/LSMInvertedIndexWorkerFactory.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/multithread/LSMInvertedIndexWorkerFactory.java
@@ -15,6 +15,7 @@
 
 package edu.uci.ics.hyracks.storage.am.lsm.invertedindex.multithread;
 
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.storage.am.common.AbstractIndexTestWorker;
 import edu.uci.ics.hyracks.storage.am.common.IIndexTestWorkerFactory;
 import edu.uci.ics.hyracks.storage.am.common.TestOperationSelector;
@@ -23,8 +24,8 @@
 
 public class LSMInvertedIndexWorkerFactory implements IIndexTestWorkerFactory {
     @Override
-    public AbstractIndexTestWorker create(DataGenThread dataGen, TestOperationSelector opSelector,
-            IIndex index, int numBatches) {
+    public AbstractIndexTestWorker create(DataGenThread dataGen, TestOperationSelector opSelector, IIndex index,
+            int numBatches) throws HyracksDataException {
         return new LSMInvertedIndexTestWorker(dataGen, opSelector, index, numBatches);
     }
 }
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndexBulkLoadTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndexBulkLoadTest.java
index fd90ad4..776b193 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndexBulkLoadTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndexBulkLoadTest.java
@@ -21,6 +21,6 @@
 public class OnDiskInvertedIndexBulkLoadTest extends AbstractInvertedIndexLoadTest {
 
     public OnDiskInvertedIndexBulkLoadTest() {
-        super(InvertedIndexType.ONDISK, true, 1);
+        super(InvertedIndexType.ONDISK, true);
     }
 }
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/ondisk/PartitionedOnDiskInvertedIndexBulkLoadTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/ondisk/PartitionedOnDiskInvertedIndexBulkLoadTest.java
index ad2b93a..6b9890d 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/ondisk/PartitionedOnDiskInvertedIndexBulkLoadTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/ondisk/PartitionedOnDiskInvertedIndexBulkLoadTest.java
@@ -21,6 +21,6 @@
 public class PartitionedOnDiskInvertedIndexBulkLoadTest extends AbstractInvertedIndexLoadTest {
 
     public PartitionedOnDiskInvertedIndexBulkLoadTest() {
-        super(InvertedIndexType.PARTITIONED_ONDISK, true, 1);
+        super(InvertedIndexType.PARTITIONED_ONDISK, true);
     }
 }
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/util/LSMInvertedIndexTestContext.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/util/LSMInvertedIndexTestContext.java
index 119226e..05dc1b5 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/util/LSMInvertedIndexTestContext.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/util/LSMInvertedIndexTestContext.java
@@ -63,7 +63,7 @@
 
     public LSMInvertedIndexTestContext(ISerializerDeserializer[] fieldSerdes, IIndex index,
             IBinaryTokenizerFactory tokenizerFactory, InvertedIndexType invIndexType,
-            InvertedIndexTokenizingTupleIterator indexTupleIter) {
+            InvertedIndexTokenizingTupleIterator indexTupleIter) throws HyracksDataException {
         super(fieldSerdes, index);
         invIndex = (IInvertedIndex) index;
         this.tokenizerFactory = tokenizerFactory;
@@ -97,7 +97,7 @@
 
     public static LSMInvertedIndexTestContext create(LSMInvertedIndexTestHarness harness,
             ISerializerDeserializer[] fieldSerdes, int tokenFieldCount, IBinaryTokenizerFactory tokenizerFactory,
-            InvertedIndexType invIndexType) throws IndexException {
+            InvertedIndexType invIndexType) throws IndexException, HyracksDataException {
         ITypeTraits[] allTypeTraits = SerdeUtils.serdesToTypeTraits(fieldSerdes);
         IBinaryComparatorFactory[] allCmpFactories = SerdeUtils.serdesToComparatorFactories(fieldSerdes,
                 fieldSerdes.length);
@@ -147,19 +147,19 @@
             case LSM: {
                 invIndex = InvertedIndexUtils.createLSMInvertedIndex(harness.getVirtualBufferCache(),
                         harness.getDiskFileMapProvider(), invListTypeTraits, invListCmpFactories, tokenTypeTraits,
-                        tokenCmpFactories, tokenizerFactory, harness.getDiskBufferCache(), harness.getIOManager(),
-                        harness.getOnDiskDir(), harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
+                        tokenCmpFactories, tokenizerFactory, harness.getDiskBufferCache(), harness.getOnDiskDir(),
+                        harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
                         harness.getOperationTracker(), harness.getIOScheduler(),
-                        harness.getIOOperationCallbackProvider(), harness.getIODeviceId());
+                        harness.getIOOperationCallbackProvider());
                 break;
             }
             case PARTITIONED_LSM: {
                 invIndex = InvertedIndexUtils.createPartitionedLSMInvertedIndex(harness.getVirtualBufferCache(),
                         harness.getDiskFileMapProvider(), invListTypeTraits, invListCmpFactories, tokenTypeTraits,
-                        tokenCmpFactories, tokenizerFactory, harness.getDiskBufferCache(), harness.getIOManager(),
-                        harness.getOnDiskDir(), harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
+                        tokenCmpFactories, tokenizerFactory, harness.getDiskBufferCache(), harness.getOnDiskDir(),
+                        harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
                         harness.getOperationTracker(), harness.getIOScheduler(),
-                        harness.getIOOperationCallbackProvider(), harness.getIODeviceId());
+                        harness.getIOOperationCallbackProvider());
                 break;
             }
             default: {
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/util/LSMInvertedIndexTestUtils.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/util/LSMInvertedIndexTestUtils.java
index fcbf133..c4e1ac5 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/util/LSMInvertedIndexTestUtils.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/util/LSMInvertedIndexTestUtils.java
@@ -205,7 +205,7 @@
         ISerializerDeserializer[] fieldSerdes = testCtx.getFieldSerdes();
 
         // Use the expected index to bulk-load the actual index.
-        IIndexBulkLoader bulkLoader = testCtx.getIndex().createBulkLoader(1.0f, false, numDocs);
+        IIndexBulkLoader bulkLoader = testCtx.getIndex().createBulkLoader(1.0f, false, numDocs, true);
         ArrayTupleBuilder tupleBuilder = new ArrayTupleBuilder(testCtx.getFieldSerdes().length);
         ArrayTupleReference tuple = new ArrayTupleReference();
         Iterator<CheckTuple> checkTupleIter = tmpMemIndex.iterator();
@@ -493,7 +493,6 @@
         int[] fieldPermutation = new int[] { 0 };
         PermutingTupleReference searchDocument = new PermutingTupleReference(fieldPermutation);
 
-        IIndexCursor resultCursor = accessor.createSearchCursor();
         int numQueries = numDocQueries + numRandomQueries;
         for (int i = 0; i < numQueries; i++) {
             // If number of documents in the corpus is less than numDocQueries, then replace the remaining ones with random queries.
@@ -511,7 +510,7 @@
             searchPred.setQueryTuple(searchDocument);
             searchPred.setQueryFieldIndex(0);
 
-            resultCursor.reset();
+            IIndexCursor resultCursor = accessor.createSearchCursor();
             boolean panic = false;
             try {
                 accessor.search(resultCursor, searchPred);
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeBulkLoadTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeBulkLoadTest.java
index b9f652a..3fcb4e5 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeBulkLoadTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeBulkLoadTest.java
@@ -35,7 +35,7 @@
 public class LSMRTreeBulkLoadTest extends AbstractRTreeBulkLoadTest {
 
     public LSMRTreeBulkLoadTest() {
-        super(1, AccessMethodTestsConfig.LSM_RTREE_TEST_RSTAR_POLICY);
+        super(AccessMethodTestsConfig.LSM_RTREE_TEST_RSTAR_POLICY);
     }
 
     private final LSMRTreeTestHarness harness = new LSMRTreeTestHarness();
@@ -54,12 +54,10 @@
     protected AbstractRTreeTestContext createTestContext(ISerializerDeserializer[] fieldSerdes,
             IPrimitiveValueProviderFactory[] valueProviderFactories, int numKeys, RTreePolicyType rtreePolicyType)
             throws Exception {
-        return LSMRTreeTestContext.create(harness.getVirtualBufferCache(), harness.getIOManager(),
-                harness.getFileReference(), harness.getDiskBufferCache(), harness.getDiskFileMapProvider(),
-                fieldSerdes, valueProviderFactories, numKeys, rtreePolicyType,
-                harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
-                harness.getOperationTracker(), harness.getIOScheduler(),
-                harness.getIOOperationCallbackProvider(), harness.getIODeviceId());
+        return LSMRTreeTestContext.create(harness.getVirtualBufferCache(), harness.getFileReference(),
+                harness.getDiskBufferCache(), harness.getDiskFileMapProvider(), fieldSerdes, valueProviderFactories,
+                numKeys, rtreePolicyType, harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
+                harness.getOperationTracker(), harness.getIOScheduler(), harness.getIOOperationCallbackProvider());
     }
 
     @Override
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeDeleteTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeDeleteTest.java
index c4c5d00..467916e 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeDeleteTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeDeleteTest.java
@@ -54,12 +54,10 @@
     protected AbstractRTreeTestContext createTestContext(ISerializerDeserializer[] fieldSerdes,
             IPrimitiveValueProviderFactory[] valueProviderFactories, int numKeys, RTreePolicyType rtreePolicyType)
             throws Exception {
-        return LSMRTreeTestContext.create(harness.getVirtualBufferCache(), harness.getIOManager(),
-                harness.getFileReference(), harness.getDiskBufferCache(), harness.getDiskFileMapProvider(),
-                fieldSerdes, valueProviderFactories, numKeys, rtreePolicyType,
-                harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
-                harness.getOperationTracker(), harness.getIOScheduler(),
-                harness.getIOOperationCallbackProvider(), harness.getIODeviceId());
+        return LSMRTreeTestContext.create(harness.getVirtualBufferCache(), harness.getFileReference(),
+                harness.getDiskBufferCache(), harness.getDiskFileMapProvider(), fieldSerdes, valueProviderFactories,
+                numKeys, rtreePolicyType, harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
+                harness.getOperationTracker(), harness.getIOScheduler(), harness.getIOOperationCallbackProvider());
     }
 
     @Override
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeExamplesTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeExamplesTest.java
index 0778269..a35170c 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeExamplesTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeExamplesTest.java
@@ -37,13 +37,12 @@
     protected ITreeIndex createTreeIndex(ITypeTraits[] typeTraits, IBinaryComparatorFactory[] rtreeCmpFactories,
             IBinaryComparatorFactory[] btreeCmpFactories, IPrimitiveValueProviderFactory[] valueProviderFactories,
             RTreePolicyType rtreePolicyType) throws TreeIndexException {
-        return LSMRTreeUtils.createLSMTree(harness.getVirtualBufferCache(), harness.getIOManager(),
-                harness.getFileReference(), harness.getDiskBufferCache(), harness.getDiskFileMapProvider(), typeTraits,
-                rtreeCmpFactories, btreeCmpFactories, valueProviderFactories, rtreePolicyType,
-                harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
-                harness.getOperationTracker(), harness.getIOScheduler(),
+        return LSMRTreeUtils.createLSMTree(harness.getVirtualBufferCache(), harness.getFileReference(),
+                harness.getDiskBufferCache(), harness.getDiskFileMapProvider(), typeTraits, rtreeCmpFactories,
+                btreeCmpFactories, valueProviderFactories, rtreePolicyType, harness.getBoomFilterFalsePositiveRate(),
+                harness.getMergePolicy(), harness.getOperationTracker(), harness.getIOScheduler(),
                 harness.getIOOperationCallbackProvider(),
-                LSMRTreeUtils.proposeBestLinearizer(typeTraits, rtreeCmpFactories.length), harness.getIODeviceId());
+                LSMRTreeUtils.proposeBestLinearizer(typeTraits, rtreeCmpFactories.length));
     }
 
     @Before
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeInsertTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeInsertTest.java
index 33e0bca..44da0c0 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeInsertTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeInsertTest.java
@@ -54,12 +54,10 @@
     protected AbstractRTreeTestContext createTestContext(ISerializerDeserializer[] fieldSerdes,
             IPrimitiveValueProviderFactory[] valueProviderFactories, int numKeys, RTreePolicyType rtreePolicyType)
             throws Exception {
-        return LSMRTreeTestContext.create(harness.getVirtualBufferCache(), harness.getIOManager(),
-                harness.getFileReference(), harness.getDiskBufferCache(), harness.getDiskFileMapProvider(),
-                fieldSerdes, valueProviderFactories, numKeys, rtreePolicyType,
-                harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
-                harness.getOperationTracker(), harness.getIOScheduler(),
-                harness.getIOOperationCallbackProvider(), harness.getIODeviceId());
+        return LSMRTreeTestContext.create(harness.getVirtualBufferCache(), harness.getFileReference(),
+                harness.getDiskBufferCache(), harness.getDiskFileMapProvider(), fieldSerdes, valueProviderFactories,
+                numKeys, rtreePolicyType, harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
+                harness.getOperationTracker(), harness.getIOScheduler(), harness.getIOOperationCallbackProvider());
     }
 
     @Override
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeLifecycleTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeLifecycleTest.java
index 5da1fe0..b97bebb 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeLifecycleTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeLifecycleTest.java
@@ -16,7 +16,6 @@
 
 import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
 import edu.uci.ics.hyracks.api.io.FileReference;
-import edu.uci.ics.hyracks.api.io.IODeviceHandle;
 import edu.uci.ics.hyracks.data.std.primitive.IntegerPointable;
 import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
 import edu.uci.ics.hyracks.storage.am.common.AbstractIndexLifecycleTest;
@@ -50,10 +49,8 @@
     @Override
     protected boolean persistentStateExists() throws Exception {
         // make sure all of the directories exist
-        for (IODeviceHandle handle : harness.getIOManager().getIODevices()) {
-            if (!new FileReference(handle, harness.getFileReference().getFile().getPath()).getFile().exists()) {
-                return false;
-            }
+        if (!new FileReference(harness.getFileReference().getFile()).getFile().exists()) {
+            return false;
         }
         return true;
     }
@@ -66,11 +63,10 @@
     @Override
     public void setup() throws Exception {
         harness.setUp();
-        testCtx = LSMRTreeTestContext.create(harness.getVirtualBufferCache(), harness.getIOManager(),
-                harness.getFileReference(), harness.getDiskBufferCache(), harness.getDiskFileMapProvider(),
-                fieldSerdes, valueProviderFactories, numKeys, RTreePolicyType.RTREE,
-                harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(), harness.getOperationTracker(),
-                harness.getIOScheduler(), harness.getIOOperationCallbackProvider(), harness.getIODeviceId());
+        testCtx = LSMRTreeTestContext.create(harness.getVirtualBufferCache(), harness.getFileReference(),
+                harness.getDiskBufferCache(), harness.getDiskFileMapProvider(), fieldSerdes, valueProviderFactories,
+                numKeys, RTreePolicyType.RTREE, harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
+                harness.getOperationTracker(), harness.getIOScheduler(), harness.getIOOperationCallbackProvider());
         index = testCtx.getIndex();
     }
 
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeMergeTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeMergeTest.java
index 42fde48..b13ce4d 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeMergeTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeMergeTest.java
@@ -53,12 +53,10 @@
     protected AbstractRTreeTestContext createTestContext(ISerializerDeserializer[] fieldSerdes,
             IPrimitiveValueProviderFactory[] valueProviderFactories, int numKeys, RTreePolicyType rtreePolicyType)
             throws Exception {
-        return LSMRTreeTestContext.create(harness.getVirtualBufferCache(), harness.getIOManager(),
-                harness.getFileReference(), harness.getDiskBufferCache(), harness.getDiskFileMapProvider(),
-                fieldSerdes, valueProviderFactories, numKeys, rtreePolicyType,
-                harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
-                harness.getOperationTracker(), harness.getIOScheduler(),
-                harness.getIOOperationCallbackProvider(), harness.getIODeviceId());
+        return LSMRTreeTestContext.create(harness.getVirtualBufferCache(), harness.getFileReference(),
+                harness.getDiskBufferCache(), harness.getDiskFileMapProvider(), fieldSerdes, valueProviderFactories,
+                numKeys, rtreePolicyType, harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
+                harness.getOperationTracker(), harness.getIOScheduler(), harness.getIOOperationCallbackProvider());
     }
 
     @Override
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeMergeTestDriver.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeMergeTestDriver.java
index 4cde8d2..18528c4 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeMergeTestDriver.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeMergeTestDriver.java
@@ -59,9 +59,15 @@
         for (int i = 0; i < maxTreesToMerge; i++) {
             for (int j = 0; j < i; j++) {
                 if (fieldSerdes[0] instanceof IntegerSerializerDeserializer) {
-                    rTreeTestUtils.bulkLoadIntTuples(ctx, numTuplesToInsert, getRandom());
+                    rTreeTestUtils.insertIntTuples(ctx, numTuplesToInsert, getRandom());
+                    // Deactivate and the re-activate the index to force it flush its in memory component
+                    ctx.getIndex().deactivate();
+                    ctx.getIndex().activate();
                 } else if (fieldSerdes[0] instanceof DoubleSerializerDeserializer) {
-                    rTreeTestUtils.bulkLoadDoubleTuples(ctx, numTuplesToInsert, getRandom());
+                    rTreeTestUtils.insertDoubleTuples(ctx, numTuplesToInsert, getRandom());
+                    // Deactivate and the re-activate the index to force it flush its in memory component
+                    ctx.getIndex().deactivate();
+                    ctx.getIndex().activate();
                 }
             }
 
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeMultiBulkLoadTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeMultiBulkLoadTest.java
deleted file mode 100644
index 6796546..0000000
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeMultiBulkLoadTest.java
+++ /dev/null
@@ -1,69 +0,0 @@
-/*
- * Copyright 2009-2013 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package edu.uci.ics.hyracks.storage.am.lsm.rtree;
-
-import java.util.Random;
-
-import org.junit.After;
-import org.junit.Before;
-
-import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
-import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
-import edu.uci.ics.hyracks.api.exceptions.HyracksException;
-import edu.uci.ics.hyracks.storage.am.common.api.IPrimitiveValueProviderFactory;
-import edu.uci.ics.hyracks.storage.am.config.AccessMethodTestsConfig;
-import edu.uci.ics.hyracks.storage.am.lsm.rtree.util.LSMRTreeTestContext;
-import edu.uci.ics.hyracks.storage.am.lsm.rtree.util.LSMRTreeTestHarness;
-import edu.uci.ics.hyracks.storage.am.rtree.AbstractRTreeBulkLoadTest;
-import edu.uci.ics.hyracks.storage.am.rtree.AbstractRTreeTestContext;
-import edu.uci.ics.hyracks.storage.am.rtree.frames.RTreePolicyType;
-
-@SuppressWarnings("rawtypes")
-public class LSMRTreeMultiBulkLoadTest extends AbstractRTreeBulkLoadTest {
-
-    public LSMRTreeMultiBulkLoadTest() {
-        super(AccessMethodTestsConfig.LSM_RTREE_BULKLOAD_ROUNDS, AccessMethodTestsConfig.LSM_RTREE_TEST_RSTAR_POLICY);
-    }
-
-    private final LSMRTreeTestHarness harness = new LSMRTreeTestHarness();
-
-    @Before
-    public void setUp() throws HyracksException {
-        harness.setUp();
-    }
-
-    @After
-    public void tearDown() throws HyracksDataException {
-        harness.tearDown();
-    }
-
-    @Override
-    protected AbstractRTreeTestContext createTestContext(ISerializerDeserializer[] fieldSerdes,
-            IPrimitiveValueProviderFactory[] valueProviderFactories, int numKeys, RTreePolicyType rtreePolicyType)
-            throws Exception {
-        return LSMRTreeTestContext.create(harness.getVirtualBufferCache(), harness.getIOManager(),
-                harness.getFileReference(), harness.getDiskBufferCache(), harness.getDiskFileMapProvider(),
-                fieldSerdes, valueProviderFactories, numKeys, rtreePolicyType,
-                harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
-                harness.getOperationTracker(), harness.getIOScheduler(),
-                harness.getIOOperationCallbackProvider(), harness.getIODeviceId());
-    }
-
-    @Override
-    protected Random getRandom() {
-        return harness.getRandom();
-    }
-}
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesBulkLoadTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesBulkLoadTest.java
index cf33d05..2baf485 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesBulkLoadTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesBulkLoadTest.java
@@ -35,7 +35,7 @@
 public class LSMRTreeWithAntiMatterTuplesBulkLoadTest extends AbstractRTreeBulkLoadTest {
 
     public LSMRTreeWithAntiMatterTuplesBulkLoadTest() {
-        super(1, AccessMethodTestsConfig.LSM_RTREE_TEST_RSTAR_POLICY);
+        super(AccessMethodTestsConfig.LSM_RTREE_TEST_RSTAR_POLICY);
     }
 
     private final LSMRTreeTestHarness harness = new LSMRTreeTestHarness();
@@ -54,11 +54,10 @@
     protected AbstractRTreeTestContext createTestContext(ISerializerDeserializer[] fieldSerdes,
             IPrimitiveValueProviderFactory[] valueProviderFactories, int numKeys, RTreePolicyType rtreePolicyType)
             throws Exception {
-        return LSMRTreeWithAntiMatterTuplesTestContext.create(harness.getVirtualBufferCache(), harness.getIOManager(),
+        return LSMRTreeWithAntiMatterTuplesTestContext.create(harness.getVirtualBufferCache(),
                 harness.getFileReference(), harness.getDiskBufferCache(), harness.getDiskFileMapProvider(),
                 fieldSerdes, valueProviderFactories, numKeys, rtreePolicyType, harness.getMergePolicy(),
-                harness.getOperationTracker(), harness.getIOScheduler(),
-                harness.getIOOperationCallbackProvider(), harness.getIODeviceId());
+                harness.getOperationTracker(), harness.getIOScheduler(), harness.getIOOperationCallbackProvider());
     }
 
     @Override
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesDeleteTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesDeleteTest.java
index 7be02f3..9c440cc 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesDeleteTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesDeleteTest.java
@@ -54,11 +54,10 @@
     protected AbstractRTreeTestContext createTestContext(ISerializerDeserializer[] fieldSerdes,
             IPrimitiveValueProviderFactory[] valueProviderFactories, int numKeys, RTreePolicyType rtreePolicyType)
             throws Exception {
-        return LSMRTreeWithAntiMatterTuplesTestContext.create(harness.getVirtualBufferCache(), harness.getIOManager(),
+        return LSMRTreeWithAntiMatterTuplesTestContext.create(harness.getVirtualBufferCache(),
                 harness.getFileReference(), harness.getDiskBufferCache(), harness.getDiskFileMapProvider(),
                 fieldSerdes, valueProviderFactories, numKeys, rtreePolicyType, harness.getMergePolicy(),
-                harness.getOperationTracker(), harness.getIOScheduler(),
-                harness.getIOOperationCallbackProvider(), harness.getIODeviceId());
+                harness.getOperationTracker(), harness.getIOScheduler(), harness.getIOOperationCallbackProvider());
     }
 
     @Override
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesExamplesTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesExamplesTest.java
index 71d6c1c..909d0cd 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesExamplesTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesExamplesTest.java
@@ -37,12 +37,12 @@
     protected ITreeIndex createTreeIndex(ITypeTraits[] typeTraits, IBinaryComparatorFactory[] rtreeCmpFactories,
             IBinaryComparatorFactory[] btreeCmpFactories, IPrimitiveValueProviderFactory[] valueProviderFactories,
             RTreePolicyType rtreePolicyType) throws TreeIndexException {
-        return LSMRTreeUtils.createLSMTreeWithAntiMatterTuples(harness.getVirtualBufferCache(), harness.getIOManager(),
+        return LSMRTreeUtils.createLSMTreeWithAntiMatterTuples(harness.getVirtualBufferCache(),
                 harness.getFileReference(), harness.getDiskBufferCache(), harness.getDiskFileMapProvider(), typeTraits,
                 rtreeCmpFactories, btreeCmpFactories, valueProviderFactories, rtreePolicyType,
                 harness.getMergePolicy(), harness.getOperationTracker(), harness.getIOScheduler(),
                 harness.getIOOperationCallbackProvider(),
-                LSMRTreeUtils.proposeBestLinearizer(typeTraits, rtreeCmpFactories.length), harness.getIODeviceId());
+                LSMRTreeUtils.proposeBestLinearizer(typeTraits, rtreeCmpFactories.length));
     }
 
     @Before
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesInsertTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesInsertTest.java
index ad502c7..6b35192 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesInsertTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesInsertTest.java
@@ -54,11 +54,10 @@
     protected AbstractRTreeTestContext createTestContext(ISerializerDeserializer[] fieldSerdes,
             IPrimitiveValueProviderFactory[] valueProviderFactories, int numKeys, RTreePolicyType rtreePolicyType)
             throws Exception {
-        return LSMRTreeWithAntiMatterTuplesTestContext.create(harness.getVirtualBufferCache(), harness.getIOManager(),
+        return LSMRTreeWithAntiMatterTuplesTestContext.create(harness.getVirtualBufferCache(),
                 harness.getFileReference(), harness.getDiskBufferCache(), harness.getDiskFileMapProvider(),
                 fieldSerdes, valueProviderFactories, numKeys, rtreePolicyType, harness.getMergePolicy(),
-                harness.getOperationTracker(), harness.getIOScheduler(),
-                harness.getIOOperationCallbackProvider(), harness.getIODeviceId());
+                harness.getOperationTracker(), harness.getIOScheduler(), harness.getIOOperationCallbackProvider());
     }
 
     @Override
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesLifecycleTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesLifecycleTest.java
index 44c1966..6dcbdf2 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesLifecycleTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesLifecycleTest.java
@@ -16,7 +16,6 @@
 
 import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
 import edu.uci.ics.hyracks.api.io.FileReference;
-import edu.uci.ics.hyracks.api.io.IODeviceHandle;
 import edu.uci.ics.hyracks.data.std.primitive.IntegerPointable;
 import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
 import edu.uci.ics.hyracks.storage.am.common.AbstractIndexLifecycleTest;
@@ -50,10 +49,8 @@
     @Override
     protected boolean persistentStateExists() throws Exception {
         // make sure all of the directories exist
-        for (IODeviceHandle handle : harness.getIOManager().getIODevices()) {
-            if (!new FileReference(handle, harness.getFileReference().getFile().getPath()).getFile().exists()) {
-                return false;
-            }
+        if (!new FileReference(harness.getFileReference().getFile()).getFile().exists()) {
+            return false;
         }
         return true;
     }
@@ -67,10 +64,9 @@
     public void setup() throws Exception {
         harness.setUp();
         testCtx = LSMRTreeWithAntiMatterTuplesTestContext.create(harness.getVirtualBufferCache(),
-                harness.getIOManager(), harness.getFileReference(), harness.getDiskBufferCache(),
-                harness.getDiskFileMapProvider(), fieldSerdes, valueProviderFactories, numKeys, RTreePolicyType.RTREE,
-                harness.getMergePolicy(), harness.getOperationTracker(), harness.getIOScheduler(),
-                harness.getIOOperationCallbackProvider(), harness.getIODeviceId());
+                harness.getFileReference(), harness.getDiskBufferCache(), harness.getDiskFileMapProvider(),
+                fieldSerdes, valueProviderFactories, numKeys, RTreePolicyType.RTREE, harness.getMergePolicy(),
+                harness.getOperationTracker(), harness.getIOScheduler(), harness.getIOOperationCallbackProvider());
         index = testCtx.getIndex();
     }
 
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesMergeTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesMergeTest.java
index 16a9679..dc81fc3 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesMergeTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesMergeTest.java
@@ -53,11 +53,10 @@
     protected AbstractRTreeTestContext createTestContext(ISerializerDeserializer[] fieldSerdes,
             IPrimitiveValueProviderFactory[] valueProviderFactories, int numKeys, RTreePolicyType rtreePolicyType)
             throws Exception {
-        return LSMRTreeWithAntiMatterTuplesTestContext.create(harness.getVirtualBufferCache(), harness.getIOManager(),
+        return LSMRTreeWithAntiMatterTuplesTestContext.create(harness.getVirtualBufferCache(),
                 harness.getFileReference(), harness.getDiskBufferCache(), harness.getDiskFileMapProvider(),
                 fieldSerdes, valueProviderFactories, numKeys, rtreePolicyType, harness.getMergePolicy(),
-                harness.getOperationTracker(), harness.getIOScheduler(),
-                harness.getIOOperationCallbackProvider(), harness.getIODeviceId());
+                harness.getOperationTracker(), harness.getIOScheduler(), harness.getIOOperationCallbackProvider());
     }
 
     @Override
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesMultiBulkLoadTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesMultiBulkLoadTest.java
deleted file mode 100644
index c91948e..0000000
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesMultiBulkLoadTest.java
+++ /dev/null
@@ -1,68 +0,0 @@
-/*
- * Copyright 2009-2013 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package edu.uci.ics.hyracks.storage.am.lsm.rtree;
-
-import java.util.Random;
-
-import org.junit.After;
-import org.junit.Before;
-
-import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
-import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
-import edu.uci.ics.hyracks.api.exceptions.HyracksException;
-import edu.uci.ics.hyracks.storage.am.common.api.IPrimitiveValueProviderFactory;
-import edu.uci.ics.hyracks.storage.am.config.AccessMethodTestsConfig;
-import edu.uci.ics.hyracks.storage.am.lsm.rtree.util.LSMRTreeTestHarness;
-import edu.uci.ics.hyracks.storage.am.lsm.rtree.util.LSMRTreeWithAntiMatterTuplesTestContext;
-import edu.uci.ics.hyracks.storage.am.rtree.AbstractRTreeBulkLoadTest;
-import edu.uci.ics.hyracks.storage.am.rtree.AbstractRTreeTestContext;
-import edu.uci.ics.hyracks.storage.am.rtree.frames.RTreePolicyType;
-
-@SuppressWarnings("rawtypes")
-public class LSMRTreeWithAntiMatterTuplesMultiBulkLoadTest extends AbstractRTreeBulkLoadTest {
-
-    public LSMRTreeWithAntiMatterTuplesMultiBulkLoadTest() {
-        super(AccessMethodTestsConfig.LSM_RTREE_BULKLOAD_ROUNDS, AccessMethodTestsConfig.LSM_RTREE_TEST_RSTAR_POLICY);
-    }
-
-    private final LSMRTreeTestHarness harness = new LSMRTreeTestHarness();
-
-    @Before
-    public void setUp() throws HyracksException {
-        harness.setUp();
-    }
-
-    @After
-    public void tearDown() throws HyracksDataException {
-        harness.tearDown();
-    }
-
-    @Override
-    protected AbstractRTreeTestContext createTestContext(ISerializerDeserializer[] fieldSerdes,
-            IPrimitiveValueProviderFactory[] valueProviderFactories, int numKeys, RTreePolicyType rtreePolicyType)
-            throws Exception {
-        return LSMRTreeWithAntiMatterTuplesTestContext.create(harness.getVirtualBufferCache(), harness.getIOManager(),
-                harness.getFileReference(), harness.getDiskBufferCache(), harness.getDiskFileMapProvider(),
-                fieldSerdes, valueProviderFactories, numKeys, rtreePolicyType, harness.getMergePolicy(),
-                harness.getOperationTracker(), harness.getIOScheduler(),
-                harness.getIOOperationCallbackProvider(), harness.getIODeviceId());
-    }
-
-    @Override
-    protected Random getRandom() {
-        return harness.getRandom();
-    }
-}
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/multithread/AbstractLSMRTreeTestWorker.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/multithread/AbstractLSMRTreeTestWorker.java
index 8bf2edd..710d181 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/multithread/AbstractLSMRTreeTestWorker.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/multithread/AbstractLSMRTreeTestWorker.java
@@ -36,7 +36,7 @@
     protected final ArrayTupleReference rearrangedTuple = new ArrayTupleReference();
 
     public AbstractLSMRTreeTestWorker(DataGenThread dataGen, TestOperationSelector opSelector, IIndex index,
-            int numBatches) {
+            int numBatches) throws HyracksDataException {
         super(dataGen, opSelector, index, numBatches);
         lsmRTree = (ITreeIndex) index;
         numFields = lsmRTree.getFieldCount();
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/multithread/LSMRTreeMultiThreadTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/multithread/LSMRTreeMultiThreadTest.java
index 8c6c305..850d540 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/multithread/LSMRTreeMultiThreadTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/multithread/LSMRTreeMultiThreadTest.java
@@ -57,13 +57,12 @@
     protected ITreeIndex createTreeIndex(ITypeTraits[] typeTraits, IBinaryComparatorFactory[] rtreeCmpFactories,
             IBinaryComparatorFactory[] btreeCmpFactories, IPrimitiveValueProviderFactory[] valueProviderFactories,
             RTreePolicyType rtreePolicyType) throws TreeIndexException {
-        return LSMRTreeUtils.createLSMTree(harness.getVirtualBufferCache(), harness.getIOManager(),
-                harness.getFileReference(), harness.getDiskBufferCache(), harness.getDiskFileMapProvider(), typeTraits,
-                rtreeCmpFactories, btreeCmpFactories, valueProviderFactories, rtreePolicyType,
-                harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
-                harness.getOperationTracker(), harness.getIOScheduler(),
+        return LSMRTreeUtils.createLSMTree(harness.getVirtualBufferCache(), harness.getFileReference(),
+                harness.getDiskBufferCache(), harness.getDiskFileMapProvider(), typeTraits, rtreeCmpFactories,
+                btreeCmpFactories, valueProviderFactories, rtreePolicyType, harness.getBoomFilterFalsePositiveRate(),
+                harness.getMergePolicy(), harness.getOperationTracker(), harness.getIOScheduler(),
                 harness.getIOOperationCallbackProvider(),
-                LSMRTreeUtils.proposeBestLinearizer(typeTraits, rtreeCmpFactories.length), harness.getIODeviceId());
+                LSMRTreeUtils.proposeBestLinearizer(typeTraits, rtreeCmpFactories.length));
     }
 
     @Override
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/multithread/LSMRTreeTestWorker.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/multithread/LSMRTreeTestWorker.java
index 09de3a3..b3fddc8 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/multithread/LSMRTreeTestWorker.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/multithread/LSMRTreeTestWorker.java
@@ -39,7 +39,8 @@
     private final ArrayTupleBuilder rearrangedTb;
     private final ArrayTupleReference rearrangedTuple = new ArrayTupleReference();
 
-    public LSMRTreeTestWorker(DataGenThread dataGen, TestOperationSelector opSelector, IIndex index, int numBatches) {
+    public LSMRTreeTestWorker(DataGenThread dataGen, TestOperationSelector opSelector, IIndex index, int numBatches)
+            throws HyracksDataException {
         super(dataGen, opSelector, index, numBatches);
         lsmRTree = (LSMRTree) index;
         numFields = lsmRTree.getFieldCount();
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/multithread/LSMRTreeTestWorkerFactory.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/multithread/LSMRTreeTestWorkerFactory.java
index 2b2d72c..2df1747 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/multithread/LSMRTreeTestWorkerFactory.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/multithread/LSMRTreeTestWorkerFactory.java
@@ -15,6 +15,7 @@
 
 package edu.uci.ics.hyracks.storage.am.lsm.rtree.multithread;
 
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.storage.am.common.AbstractIndexTestWorker;
 import edu.uci.ics.hyracks.storage.am.common.IIndexTestWorkerFactory;
 import edu.uci.ics.hyracks.storage.am.common.TestOperationSelector;
@@ -23,8 +24,8 @@
 
 public class LSMRTreeTestWorkerFactory implements IIndexTestWorkerFactory {
     @Override
-    public AbstractIndexTestWorker create(DataGenThread dataGen, TestOperationSelector opSelector,
-            IIndex index, int numBatches) {
+    public AbstractIndexTestWorker create(DataGenThread dataGen, TestOperationSelector opSelector, IIndex index,
+            int numBatches) throws HyracksDataException {
         return new LSMRTreeTestWorker(dataGen, opSelector, index, numBatches);
     }
 }
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/multithread/LSMRTreeWithAntiMatterTuplesMultiThreadTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/multithread/LSMRTreeWithAntiMatterTuplesMultiThreadTest.java
index d9e4658..4a6b462 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/multithread/LSMRTreeWithAntiMatterTuplesMultiThreadTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/multithread/LSMRTreeWithAntiMatterTuplesMultiThreadTest.java
@@ -57,12 +57,12 @@
     protected ITreeIndex createTreeIndex(ITypeTraits[] typeTraits, IBinaryComparatorFactory[] rtreeCmpFactories,
             IBinaryComparatorFactory[] btreeCmpFactories, IPrimitiveValueProviderFactory[] valueProviderFactories,
             RTreePolicyType rtreePolicyType) throws TreeIndexException {
-        return LSMRTreeUtils.createLSMTreeWithAntiMatterTuples(harness.getVirtualBufferCache(), harness.getIOManager(),
+        return LSMRTreeUtils.createLSMTreeWithAntiMatterTuples(harness.getVirtualBufferCache(),
                 harness.getFileReference(), harness.getDiskBufferCache(), harness.getDiskFileMapProvider(), typeTraits,
                 rtreeCmpFactories, btreeCmpFactories, valueProviderFactories, rtreePolicyType,
                 harness.getMergePolicy(), harness.getOperationTracker(), harness.getIOScheduler(),
                 harness.getIOOperationCallbackProvider(),
-                LSMRTreeUtils.proposeBestLinearizer(typeTraits, rtreeCmpFactories.length), harness.getIODeviceId());
+                LSMRTreeUtils.proposeBestLinearizer(typeTraits, rtreeCmpFactories.length));
 
     }
 
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/multithread/LSMRTreeWithAntiMatterTuplesTestWorker.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/multithread/LSMRTreeWithAntiMatterTuplesTestWorker.java
index c27158f..12d7742 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/multithread/LSMRTreeWithAntiMatterTuplesTestWorker.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/multithread/LSMRTreeWithAntiMatterTuplesTestWorker.java
@@ -31,7 +31,7 @@
 public class LSMRTreeWithAntiMatterTuplesTestWorker extends AbstractLSMRTreeTestWorker {
 
     public LSMRTreeWithAntiMatterTuplesTestWorker(DataGenThread dataGen, TestOperationSelector opSelector,
-            IIndex index, int numBatches) {
+            IIndex index, int numBatches) throws HyracksDataException {
         super(dataGen, opSelector, index, numBatches);
     }
 
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/multithread/LSMRTreeWithAntiMatterTuplesTestWorkerFactory.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/multithread/LSMRTreeWithAntiMatterTuplesTestWorkerFactory.java
index 67d9c27..cf3ac02 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/multithread/LSMRTreeWithAntiMatterTuplesTestWorkerFactory.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/multithread/LSMRTreeWithAntiMatterTuplesTestWorkerFactory.java
@@ -15,6 +15,7 @@
 
 package edu.uci.ics.hyracks.storage.am.lsm.rtree.multithread;
 
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.storage.am.common.AbstractIndexTestWorker;
 import edu.uci.ics.hyracks.storage.am.common.IIndexTestWorkerFactory;
 import edu.uci.ics.hyracks.storage.am.common.TestOperationSelector;
@@ -23,8 +24,8 @@
 
 public class LSMRTreeWithAntiMatterTuplesTestWorkerFactory implements IIndexTestWorkerFactory {
     @Override
-    public AbstractIndexTestWorker create(DataGenThread dataGen, TestOperationSelector opSelector,
-            IIndex index, int numBatches) {
+    public AbstractIndexTestWorker create(DataGenThread dataGen, TestOperationSelector opSelector, IIndex index,
+            int numBatches) throws HyracksDataException {
         return new LSMRTreeWithAntiMatterTuplesTestWorker(dataGen, opSelector, index, numBatches);
     }
 }
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/util/LSMRTreeTestContext.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/util/LSMRTreeTestContext.java
index fe2cc9e..8c59e31 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/util/LSMRTreeTestContext.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/util/LSMRTreeTestContext.java
@@ -20,8 +20,8 @@
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
 import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
 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.control.nc.io.IOManager;
 import edu.uci.ics.hyracks.dataflow.common.util.SerdeUtils;
 import edu.uci.ics.hyracks.storage.am.common.api.IPrimitiveValueProviderFactory;
 import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
@@ -41,7 +41,7 @@
 @SuppressWarnings("rawtypes")
 public final class LSMRTreeTestContext extends AbstractRTreeTestContext {
 
-    public LSMRTreeTestContext(ISerializerDeserializer[] fieldSerdes, ITreeIndex treeIndex) {
+    public LSMRTreeTestContext(ISerializerDeserializer[] fieldSerdes, ITreeIndex treeIndex) throws HyracksDataException {
         super(fieldSerdes, treeIndex);
     }
 
@@ -66,22 +66,21 @@
         return lsmTree.getComparatorFactories();
     }
 
-    public static LSMRTreeTestContext create(IVirtualBufferCache virtualBufferCache, IOManager ioManager,
-            FileReference file, IBufferCache diskBufferCache, IFileMapProvider diskFileMapProvider,
-            ISerializerDeserializer[] fieldSerdes, IPrimitiveValueProviderFactory[] valueProviderFactories,
-            int numKeyFields, RTreePolicyType rtreePolicyType, double bloomFilterFalsePositiveRate,
-            ILSMMergePolicy mergePolicy, ILSMOperationTracker opTracker, ILSMIOOperationScheduler ioScheduler,
-            ILSMIOOperationCallbackProvider ioOpCallbackProvider, int ioDeviceId) throws Exception {
+    public static LSMRTreeTestContext create(IVirtualBufferCache virtualBufferCache, FileReference file,
+            IBufferCache diskBufferCache, IFileMapProvider diskFileMapProvider, ISerializerDeserializer[] fieldSerdes,
+            IPrimitiveValueProviderFactory[] valueProviderFactories, int numKeyFields, RTreePolicyType rtreePolicyType,
+            double bloomFilterFalsePositiveRate, ILSMMergePolicy mergePolicy, ILSMOperationTracker opTracker,
+            ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallbackProvider ioOpCallbackProvider)
+            throws Exception {
         ITypeTraits[] typeTraits = SerdeUtils.serdesToTypeTraits(fieldSerdes);
         IBinaryComparatorFactory[] rtreeCmpFactories = SerdeUtils
                 .serdesToComparatorFactories(fieldSerdes, numKeyFields);
         IBinaryComparatorFactory[] btreeCmpFactories = SerdeUtils.serdesToComparatorFactories(fieldSerdes,
                 fieldSerdes.length);
-        LSMRTree lsmTree = LSMRTreeUtils.createLSMTree(virtualBufferCache, ioManager, file, diskBufferCache,
-                diskFileMapProvider, typeTraits, rtreeCmpFactories, btreeCmpFactories, valueProviderFactories,
-                rtreePolicyType, bloomFilterFalsePositiveRate, mergePolicy, opTracker, ioScheduler,
-                ioOpCallbackProvider, LSMRTreeUtils.proposeBestLinearizer(typeTraits, rtreeCmpFactories.length),
-                ioDeviceId);
+        LSMRTree lsmTree = LSMRTreeUtils.createLSMTree(virtualBufferCache, file, diskBufferCache, diskFileMapProvider,
+                typeTraits, rtreeCmpFactories, btreeCmpFactories, valueProviderFactories, rtreePolicyType,
+                bloomFilterFalsePositiveRate, mergePolicy, opTracker, ioScheduler, ioOpCallbackProvider,
+                LSMRTreeUtils.proposeBestLinearizer(typeTraits, rtreeCmpFactories.length));
         LSMRTreeTestContext testCtx = new LSMRTreeTestContext(fieldSerdes, lsmTree);
         return testCtx;
     }
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/util/LSMRTreeTestHarness.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/util/LSMRTreeTestHarness.java
index 3382318..118b1bc 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/util/LSMRTreeTestHarness.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/util/LSMRTreeTestHarness.java
@@ -104,35 +104,34 @@
     }
 
     public void setUp() throws HyracksException {
-        onDiskDir = "lsm_rtree_" + simpleDateFormat.format(new Date()) + sep;
+        ioManager = TestStorageManagerComponentHolder.getIOManager();
+        ioDeviceId = 0;
+        onDiskDir = ioManager.getIODevices().get(ioDeviceId).getPath() + sep + "lsm_rtree_"
+                + simpleDateFormat.format(new Date()) + sep;
         file = new FileReference(new File(onDiskDir));
         ctx = TestUtils.create(getHyracksFrameSize());
         TestStorageManagerComponentHolder.init(diskPageSize, diskNumPages, diskMaxOpenFiles);
         diskBufferCache = TestStorageManagerComponentHolder.getBufferCache(ctx);
         diskFileMapProvider = TestStorageManagerComponentHolder.getFileMapProvider(ctx);
         virtualBufferCache = new VirtualBufferCache(new HeapBufferAllocator(), memPageSize, memNumPages);
-        ioManager = TestStorageManagerComponentHolder.getIOManager();
-        ioDeviceId = 0;
         rnd.setSeed(RANDOM_SEED);
     }
 
     public void tearDown() throws HyracksDataException {
         diskBufferCache.close();
-        for (IODeviceHandle dev : ioManager.getIODevices()) {
-            File dir = new File(dev.getPath(), onDiskDir);
-            FilenameFilter filter = new FilenameFilter() {
-                public boolean accept(File dir, String name) {
-                    return !name.startsWith(".");
-                }
-            };
-            String[] files = dir.list(filter);
-            if (files != null) {
-                for (String fileName : files) {
-                    File file = new File(dir.getPath() + File.separator + fileName);
-                    file.delete();
-                }
+        IODeviceHandle dev = ioManager.getIODevices().get(ioDeviceId);
+        File dir = new File(dev.getPath(), onDiskDir);
+        FilenameFilter filter = new FilenameFilter() {
+            public boolean accept(File dir, String name) {
+                return !name.startsWith(".");
             }
-            dir.delete();
+        };
+        String[] files = dir.list(filter);
+        if (files != null) {
+            for (String fileName : files) {
+                File file = new File(dir.getPath() + File.separator + fileName);
+                file.delete();
+            }
         }
     }
 
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/util/LSMRTreeWithAntiMatterTuplesTestContext.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/util/LSMRTreeWithAntiMatterTuplesTestContext.java
index b1f4d8f..95716f6 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/util/LSMRTreeWithAntiMatterTuplesTestContext.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/util/LSMRTreeWithAntiMatterTuplesTestContext.java
@@ -20,8 +20,8 @@
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
 import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
 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.control.nc.io.IOManager;
 import edu.uci.ics.hyracks.dataflow.common.util.SerdeUtils;
 import edu.uci.ics.hyracks.storage.am.common.api.IPrimitiveValueProviderFactory;
 import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
@@ -41,7 +41,8 @@
 @SuppressWarnings("rawtypes")
 public final class LSMRTreeWithAntiMatterTuplesTestContext extends AbstractRTreeTestContext {
 
-    public LSMRTreeWithAntiMatterTuplesTestContext(ISerializerDeserializer[] fieldSerdes, ITreeIndex treeIndex) {
+    public LSMRTreeWithAntiMatterTuplesTestContext(ISerializerDeserializer[] fieldSerdes, ITreeIndex treeIndex)
+            throws HyracksDataException {
         super(fieldSerdes, treeIndex);
     }
 
@@ -67,21 +68,20 @@
     }
 
     public static LSMRTreeWithAntiMatterTuplesTestContext create(IVirtualBufferCache virtualBufferCache,
-            IOManager ioManager, FileReference file, IBufferCache diskBufferCache,
-            IFileMapProvider diskFileMapProvider, ISerializerDeserializer[] fieldSerdes,
-            IPrimitiveValueProviderFactory[] valueProviderFactories, int numKeyFields, RTreePolicyType rtreePolicyType,
-            ILSMMergePolicy mergePolicy, ILSMOperationTracker opTracker, ILSMIOOperationScheduler ioScheduler,
-            ILSMIOOperationCallbackProvider ioOpCallbackProvider, int ioDeviceId) throws Exception {
+            FileReference file, IBufferCache diskBufferCache, IFileMapProvider diskFileMapProvider,
+            ISerializerDeserializer[] fieldSerdes, IPrimitiveValueProviderFactory[] valueProviderFactories,
+            int numKeyFields, RTreePolicyType rtreePolicyType, ILSMMergePolicy mergePolicy,
+            ILSMOperationTracker opTracker, ILSMIOOperationScheduler ioScheduler,
+            ILSMIOOperationCallbackProvider ioOpCallbackProvider) throws Exception {
         ITypeTraits[] typeTraits = SerdeUtils.serdesToTypeTraits(fieldSerdes);
         IBinaryComparatorFactory[] rtreeCmpFactories = SerdeUtils
                 .serdesToComparatorFactories(fieldSerdes, numKeyFields);
         IBinaryComparatorFactory[] btreeCmpFactories = SerdeUtils.serdesToComparatorFactories(fieldSerdes,
                 fieldSerdes.length);
         LSMRTreeWithAntiMatterTuples lsmTree = LSMRTreeUtils.createLSMTreeWithAntiMatterTuples(virtualBufferCache,
-                ioManager, file, diskBufferCache, diskFileMapProvider, typeTraits, rtreeCmpFactories,
-                btreeCmpFactories, valueProviderFactories, rtreePolicyType, mergePolicy, opTracker, ioScheduler,
-                ioOpCallbackProvider, LSMRTreeUtils.proposeBestLinearizer(typeTraits, rtreeCmpFactories.length),
-                ioDeviceId);
+                file, diskBufferCache, diskFileMapProvider, typeTraits, rtreeCmpFactories, btreeCmpFactories,
+                valueProviderFactories, rtreePolicyType, mergePolicy, opTracker, ioScheduler, ioOpCallbackProvider,
+                LSMRTreeUtils.proposeBestLinearizer(typeTraits, rtreeCmpFactories.length));
         LSMRTreeWithAntiMatterTuplesTestContext testCtx = new LSMRTreeWithAntiMatterTuplesTestContext(fieldSerdes,
                 lsmTree);
         return testCtx;
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/rtree/RTreeBulkLoadTest.java b/hyracks/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/rtree/RTreeBulkLoadTest.java
index 7f42223..4b68ddc 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/rtree/RTreeBulkLoadTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/rtree/RTreeBulkLoadTest.java
@@ -32,7 +32,7 @@
 public class RTreeBulkLoadTest extends AbstractRTreeBulkLoadTest {
 
     public RTreeBulkLoadTest() {
-        super(1, AccessMethodTestsConfig.RTREE_TEST_RSTAR_POLICY);
+        super(AccessMethodTestsConfig.RTREE_TEST_RSTAR_POLICY);
     }
 
     private final RTreeTestHarness harness = new RTreeTestHarness();
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/rtree/multithread/RTreeTestWorker.java b/hyracks/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/rtree/multithread/RTreeTestWorker.java
index cf118a3..bc5d331 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/rtree/multithread/RTreeTestWorker.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/rtree/multithread/RTreeTestWorker.java
@@ -38,7 +38,8 @@
     private final ArrayTupleReference rearrangedTuple = new ArrayTupleReference();
     private final ArrayTupleBuilder rearrangedTb;
 
-    public RTreeTestWorker(DataGenThread dataGen, TestOperationSelector opSelector, IIndex index, int numBatches) {
+    public RTreeTestWorker(DataGenThread dataGen, TestOperationSelector opSelector, IIndex index, int numBatches)
+            throws HyracksDataException {
         super(dataGen, opSelector, index, numBatches);
         rtree = (RTree) index;
         numFields = rtree.getFieldCount();
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/rtree/multithread/RTreeTestWorkerFactory.java b/hyracks/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/rtree/multithread/RTreeTestWorkerFactory.java
index a1f2155..4d06601 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/rtree/multithread/RTreeTestWorkerFactory.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/rtree/multithread/RTreeTestWorkerFactory.java
@@ -15,6 +15,7 @@
 
 package edu.uci.ics.hyracks.storage.am.rtree.multithread;
 
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.storage.am.common.AbstractIndexTestWorker;
 import edu.uci.ics.hyracks.storage.am.common.IIndexTestWorkerFactory;
 import edu.uci.ics.hyracks.storage.am.common.TestOperationSelector;
@@ -23,8 +24,8 @@
 
 public class RTreeTestWorkerFactory implements IIndexTestWorkerFactory {
     @Override
-    public AbstractIndexTestWorker create(DataGenThread dataGen, TestOperationSelector opSelector,
-            IIndex index, int numBatches) {
+    public AbstractIndexTestWorker create(DataGenThread dataGen, TestOperationSelector opSelector, IIndex index,
+            int numBatches) throws HyracksDataException {
         return new RTreeTestWorker(dataGen, opSelector, index, numBatches);
     }
 }
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/rtree/utils/RTreeTestContext.java b/hyracks/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/rtree/utils/RTreeTestContext.java
index 97ac221..5968fe9 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/rtree/utils/RTreeTestContext.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/rtree/utils/RTreeTestContext.java
@@ -18,6 +18,7 @@
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
 import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
 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.dataflow.common.util.SerdeUtils;
 import edu.uci.ics.hyracks.storage.am.common.api.IPrimitiveValueProviderFactory;
@@ -32,7 +33,7 @@
 @SuppressWarnings("rawtypes")
 public class RTreeTestContext extends AbstractRTreeTestContext {
 
-    public RTreeTestContext(ISerializerDeserializer[] fieldSerdes, ITreeIndex treeIndex) {
+    public RTreeTestContext(ISerializerDeserializer[] fieldSerdes, ITreeIndex treeIndex) throws HyracksDataException {
         super(fieldSerdes, treeIndex);
     }
 
diff --git a/pregelix/pregelix-api/src/main/java/edu/uci/ics/pregelix/api/graph/Vertex.java b/pregelix/pregelix-api/src/main/java/edu/uci/ics/pregelix/api/graph/Vertex.java
index 7c91c02..4175078 100644
--- a/pregelix/pregelix-api/src/main/java/edu/uci/ics/pregelix/api/graph/Vertex.java
+++ b/pregelix/pregelix-api/src/main/java/edu/uci/ics/pregelix/api/graph/Vertex.java
@@ -420,7 +420,7 @@
     private M allocateMessage() {
         M message;
         if (usedMessage < msgPool.size()) {
-            message = msgPool.get(usedEdge);
+            message = msgPool.get(usedMessage);
             usedMessage++;
         } else {
             message = BspUtils.<M> createMessageValue(getContext().getConfiguration());
diff --git a/pregelix/pregelix-api/src/main/java/edu/uci/ics/pregelix/api/graph/VertexPartitioner.java b/pregelix/pregelix-api/src/main/java/edu/uci/ics/pregelix/api/graph/VertexPartitioner.java
new file mode 100644
index 0000000..f51ad88
--- /dev/null
+++ b/pregelix/pregelix-api/src/main/java/edu/uci/ics/pregelix/api/graph/VertexPartitioner.java
@@ -0,0 +1,36 @@
+/*
+ * 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.pregelix.api.graph;
+
+import org.apache.hadoop.io.WritableComparable;
+
+/**
+ * Users can extend this class to implement the desired vertex partitioning behavior.
+ * 
+ * @author yingyib
+ */
+@SuppressWarnings("rawtypes")
+public abstract class VertexPartitioner<I extends WritableComparable> {
+
+    /**
+     * @param vertexId
+     *            The input vertex id.
+     * @param nPartitions
+     *            The total number of partitions.
+     * @return The partition id.
+     */
+    public abstract int getPartitionId(I vertexId, int nPartitions);
+
+}
diff --git a/pregelix/pregelix-api/src/main/java/edu/uci/ics/pregelix/api/job/PregelixJob.java b/pregelix/pregelix-api/src/main/java/edu/uci/ics/pregelix/api/job/PregelixJob.java
index 81472aa..4cddaf0 100644
--- a/pregelix/pregelix-api/src/main/java/edu/uci/ics/pregelix/api/job/PregelixJob.java
+++ b/pregelix/pregelix-api/src/main/java/edu/uci/ics/pregelix/api/job/PregelixJob.java
@@ -24,6 +24,7 @@
 import edu.uci.ics.pregelix.api.graph.MessageCombiner;
 import edu.uci.ics.pregelix.api.graph.NormalizedKeyComputer;
 import edu.uci.ics.pregelix.api.graph.Vertex;
+import edu.uci.ics.pregelix.api.graph.VertexPartitioner;
 import edu.uci.ics.pregelix.api.io.VertexInputFormat;
 import edu.uci.ics.pregelix.api.io.VertexOutputFormat;
 
@@ -59,6 +60,8 @@
     public static final String FINAL_AGGREGATE_VALUE_CLASS = "pregelix.finalAggregateValueClass";
     /** The normalized key computer class */
     public static final String NMK_COMPUTER_CLASS = "pregelix.nmkComputerClass";
+    /** The partitioner class */
+    public static final String PARTITIONER_CLASS = "pregelix.partitionerClass";
     /** num of vertices */
     public static final String NUM_VERTICE = "pregelix.numVertices";
     /** num of edges */
@@ -69,6 +72,8 @@
     public static final String JOB_ID = "pregelix.jobid";
     /** frame size */
     public static final String FRAME_SIZE = "pregelix.framesize";
+    /** update intensive */
+    public static final String UPDATE_INTENSIVE = "pregelix.updateIntensive";
 
     /**
      * Constructor that will instantiate the configuration
@@ -178,4 +183,22 @@
     final public void setNoramlizedKeyComputerClass(Class<?> nkcClass) {
         getConfiguration().setClass(NMK_COMPUTER_CLASS, nkcClass, NormalizedKeyComputer.class);
     }
+
+    /**
+     * Set the vertex partitioner class
+     * 
+     * @param partitionerClass
+     */
+    final public void setVertexPartitionerClass(Class<?> partitionerClass) {
+        getConfiguration().setClass(PARTITIONER_CLASS, partitionerClass, VertexPartitioner.class);
+    }
+
+    /**
+     * Indicate if the job needs to do a lot of graph mutations or variable size updates
+     * 
+     * @param updateHeavyFlag
+     */
+    final public void setMutationOrVariableSizedUpdateHeavy(boolean variableSizedUpdateHeavyFlag) {
+        getConfiguration().setBoolean(UPDATE_INTENSIVE, variableSizedUpdateHeavyFlag);
+    }
 }
diff --git a/pregelix/pregelix-api/src/main/java/edu/uci/ics/pregelix/api/util/BspUtils.java b/pregelix/pregelix-api/src/main/java/edu/uci/ics/pregelix/api/util/BspUtils.java
index 6bac923..03c37dc 100644
--- a/pregelix/pregelix-api/src/main/java/edu/uci/ics/pregelix/api/util/BspUtils.java
+++ b/pregelix/pregelix-api/src/main/java/edu/uci/ics/pregelix/api/util/BspUtils.java
@@ -25,6 +25,7 @@
 import edu.uci.ics.pregelix.api.graph.MsgList;
 import edu.uci.ics.pregelix.api.graph.NormalizedKeyComputer;
 import edu.uci.ics.pregelix.api.graph.Vertex;
+import edu.uci.ics.pregelix.api.graph.VertexPartitioner;
 import edu.uci.ics.pregelix.api.io.VertexInputFormat;
 import edu.uci.ics.pregelix.api.io.VertexOutputFormat;
 import edu.uci.ics.pregelix.api.job.PregelixJob;
@@ -156,7 +157,7 @@
     }
 
     /**
-     * Create a global aggregator class
+     * Create a global aggregator object
      * 
      * @param conf
      *            Configuration to check
@@ -391,9 +392,9 @@
         try {
             return aggregateValueClass.newInstance();
         } catch (InstantiationException e) {
-            throw new IllegalArgumentException("createMessageValue: Failed to instantiate", e);
+            throw new IllegalArgumentException("createPartialAggregateValue: Failed to instantiate", e);
         } catch (IllegalAccessException e) {
-            throw new IllegalArgumentException("createMessageValue: Illegally accessed", e);
+            throw new IllegalArgumentException("createPartialAggregateValue: Illegally accessed", e);
         }
     }
 
@@ -415,9 +416,9 @@
             }
             return instance;
         } catch (InstantiationException e) {
-            throw new IllegalArgumentException("createMessageValue: Failed to instantiate", e);
+            throw new IllegalArgumentException("createPartialCombineValue: Failed to instantiate", e);
         } catch (IllegalAccessException e) {
-            throw new IllegalArgumentException("createMessageValue: Illegally accessed", e);
+            throw new IllegalArgumentException("createPartialCombineValue: Illegally accessed", e);
         }
     }
 
@@ -433,13 +434,46 @@
         try {
             return aggregateValueClass.newInstance();
         } catch (InstantiationException e) {
-            throw new IllegalArgumentException("createMessageValue: Failed to instantiate", e);
+            throw new IllegalArgumentException("createAggregateValue: Failed to instantiate", e);
         } catch (IllegalAccessException e) {
-            throw new IllegalArgumentException("createMessageValue: Illegally accessed", e);
+            throw new IllegalArgumentException("createAggregateValue: Illegally accessed", e);
         }
     }
 
     /**
+     * Create a user aggregate value
+     * 
+     * @param conf
+     *            Configuration to check
+     * @return Instantiated user aggregate value
+     */
+    @SuppressWarnings("rawtypes")
+    public static VertexPartitioner createVertexPartitioner(Configuration conf) {
+        Class<? extends VertexPartitioner> vertexPartitionerClass = getVertexPartitionerClass(conf);
+        try {
+            return vertexPartitionerClass.newInstance();
+        } catch (InstantiationException e) {
+            throw new IllegalArgumentException("createVertexPartitioner: Failed to instantiate", e);
+        } catch (IllegalAccessException e) {
+            throw new IllegalArgumentException("createVertexPartitioner: Illegally accessed", e);
+        }
+    }
+
+    /**
+     * Get the user's subclassed vertex partitioner class.
+     * 
+     * @param conf
+     *            Configuration to check
+     * @return The user defined vertex partitioner class
+     */
+    @SuppressWarnings({ "unchecked", "rawtypes" })
+    public static <V extends VertexPartitioner> Class<V> getVertexPartitionerClass(Configuration conf) {
+        if (conf == null)
+            conf = defaultConf;
+        return (Class<V>) conf.getClass(PregelixJob.PARTITIONER_CLASS, null, VertexPartitioner.class);
+    }
+
+    /**
      * Get the job configuration parameter whether the vertex states will increase dynamically
      * 
      * @param conf
@@ -460,4 +494,14 @@
     public static int getFrameSize(Configuration conf) {
         return conf.getInt(PregelixJob.FRAME_SIZE, -1);
     }
+
+    /**
+     * Should the job use LSM or B-tree to store vertices
+     * 
+     * @param conf
+     * @return
+     */
+    public static boolean useLSM(Configuration conf) {
+        return conf.getBoolean(PregelixJob.UPDATE_INTENSIVE, false);
+    }
 }
diff --git a/pregelix/pregelix-api/src/main/java/edu/uci/ics/pregelix/api/util/DefaultVertexPartitioner.java b/pregelix/pregelix-api/src/main/java/edu/uci/ics/pregelix/api/util/DefaultVertexPartitioner.java
new file mode 100644
index 0000000..263ec65
--- /dev/null
+++ b/pregelix/pregelix-api/src/main/java/edu/uci/ics/pregelix/api/util/DefaultVertexPartitioner.java
@@ -0,0 +1,35 @@
+/*
+ * 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.pregelix.api.util;
+
+import org.apache.hadoop.io.WritableComparable;
+
+import edu.uci.ics.pregelix.api.graph.VertexPartitioner;
+
+/**
+ * The deafult vertex partitioner which use the hashcode of the vertex id to determine the partition
+ * of the vertex.
+ * 
+ * @author yingyib
+ */
+@SuppressWarnings("rawtypes")
+public class DefaultVertexPartitioner<I extends WritableComparable> extends VertexPartitioner<I> {
+
+    @Override
+    public int getPartitionId(I vertexId, int nPartitions) {
+        return vertexId.hashCode() % nPartitions;
+    }
+
+}
diff --git a/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/hadoop/config/ConfigurationFactory.java b/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/hadoop/config/ConfigurationFactory.java
index c0bbafd..1600ab5 100644
--- a/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/hadoop/config/ConfigurationFactory.java
+++ b/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/hadoop/config/ConfigurationFactory.java
@@ -45,4 +45,15 @@
             throw new HyracksDataException(e);
         }
     }
+    
+    @Override
+    public Configuration createConfiguration() throws HyracksDataException{
+        try {
+            Configuration conf = new Configuration();
+            SerDeUtils.deserialize(conf, data);
+            return conf;
+        } catch (Exception e) {
+            throw new HyracksDataException(e);
+        }
+    }
 }
diff --git a/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGen.java b/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGen.java
index fcdf654..478ac07 100644
--- a/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGen.java
+++ b/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGen.java
@@ -53,15 +53,22 @@
 import edu.uci.ics.hyracks.storage.am.btree.dataflow.BTreeDataflowHelperFactory;
 import edu.uci.ics.hyracks.storage.am.btree.dataflow.BTreeSearchOperatorDescriptor;
 import edu.uci.ics.hyracks.storage.am.common.api.IIndexLifecycleManagerProvider;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
 import edu.uci.ics.hyracks.storage.am.common.dataflow.IndexDropOperatorDescriptor;
 import edu.uci.ics.hyracks.storage.am.common.dataflow.TreeIndexBulkLoadOperatorDescriptor;
 import edu.uci.ics.hyracks.storage.am.common.dataflow.TreeIndexCreateOperatorDescriptor;
 import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallbackFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.btree.dataflow.LSMBTreeDataflowHelperFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.ConstantMergePolicyProvider;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.NoOpIOOperationCallback;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.NoOpOperationTrackerProvider;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.SynchronousSchedulerProvider;
 import edu.uci.ics.hyracks.storage.common.IStorageManagerInterface;
 import edu.uci.ics.hyracks.storage.common.file.TransientLocalResourceFactoryProvider;
 import edu.uci.ics.pregelix.api.graph.GlobalAggregator;
 import edu.uci.ics.pregelix.api.graph.MessageCombiner;
 import edu.uci.ics.pregelix.api.graph.Vertex;
+import edu.uci.ics.pregelix.api.graph.VertexPartitioner;
 import edu.uci.ics.pregelix.api.io.VertexInputFormat;
 import edu.uci.ics.pregelix.api.job.PregelixJob;
 import edu.uci.ics.pregelix.api.util.BspUtils;
@@ -80,8 +87,10 @@
 import edu.uci.ics.pregelix.dataflow.std.base.IRuntimeHookFactory;
 import edu.uci.ics.pregelix.runtime.bootstrap.IndexLifeCycleManagerProvider;
 import edu.uci.ics.pregelix.runtime.bootstrap.StorageManagerInterface;
+import edu.uci.ics.pregelix.runtime.bootstrap.VirtualBufferCacheProvider;
 import edu.uci.ics.pregelix.runtime.touchpoint.RuntimeHookFactory;
 import edu.uci.ics.pregelix.runtime.touchpoint.VertexIdPartitionComputerFactory;
+import edu.uci.ics.pregelix.runtime.touchpoint.VertexPartitionComputerFactory;
 import edu.uci.ics.pregelix.runtime.touchpoint.WritableSerializerDeserializerFactory;
 
 public abstract class JobGen implements IJobGen {
@@ -166,10 +175,12 @@
         comparatorFactories[0] = new WritableComparingBinaryComparatorFactory(WritableComparator.get(vertexIdClass)
                 .getClass());
 
+        int[] keyFields = new int[1];
+        keyFields[0] = 0;
         IFileSplitProvider fileSplitProvider = ClusterConfig.getFileSplitProvider(jobId, PRIMARY_INDEX);
         TreeIndexCreateOperatorDescriptor btreeCreate = new TreeIndexCreateOperatorDescriptor(spec,
-                storageManagerInterface, lcManagerProvider, fileSplitProvider, typeTraits, comparatorFactories, null,
-                new BTreeDataflowHelperFactory(), new TransientLocalResourceFactoryProvider(),
+                storageManagerInterface, lcManagerProvider, fileSplitProvider, typeTraits, comparatorFactories,
+                keyFields, getIndexDataflowHelperFactory(), new TransientLocalResourceFactoryProvider(),
                 NoOpOperationCallbackFactory.INSTANCE);
         ClusterConfig.setLocationConstraint(spec, btreeCreate);
         spec.setFrameSize(frameSize);
@@ -229,16 +240,15 @@
         typeTraits[0] = new TypeTraits(false);
         typeTraits[1] = new TypeTraits(false);
         TreeIndexBulkLoadOperatorDescriptor btreeBulkLoad = new TreeIndexBulkLoadOperatorDescriptor(spec,
-                storageManagerInterface, lcManagerProvider, fileSplitProvider, typeTraits, comparatorFactories, null,
-                fieldPermutation, DEFAULT_BTREE_FILL_FACTOR, false, 0, new BTreeDataflowHelperFactory(),
-                NoOpOperationCallbackFactory.INSTANCE);
+                storageManagerInterface, lcManagerProvider, fileSplitProvider, typeTraits, comparatorFactories,
+                sortFields, fieldPermutation, DEFAULT_BTREE_FILL_FACTOR, false, 0, false,
+                getIndexDataflowHelperFactory(), NoOpOperationCallbackFactory.INSTANCE);
         ClusterConfig.setLocationConstraint(spec, btreeBulkLoad);
 
         /**
          * connect operator descriptors
          */
-        ITuplePartitionComputerFactory hashPartitionComputerFactory = new VertexIdPartitionComputerFactory(
-                new WritableSerializerDeserializerFactory(vertexIdClass));
+        ITuplePartitionComputerFactory hashPartitionComputerFactory = getVertexPartitionComputerFactory();
         spec.connect(new MToNPartitioningConnectorDescriptor(spec, hashPartitionComputerFactory), scanner, 0, sorter, 0);
         spec.connect(new OneToOneConnectorDescriptor(spec), sorter, 0, btreeBulkLoad, 0);
         spec.setFrameSize(frameSize);
@@ -313,8 +323,7 @@
         /**
          * connect operator descriptors
          */
-        ITuplePartitionComputerFactory hashPartitionComputerFactory = new VertexIdPartitionComputerFactory(
-                new WritableSerializerDeserializerFactory(vertexIdClass));
+        ITuplePartitionComputerFactory hashPartitionComputerFactory = getVertexPartitionComputerFactory();
         spec.connect(new OneToOneConnectorDescriptor(spec), scanner, 0, sorter, 0);
         spec.connect(new MToNPartitioningMergingConnectorDescriptor(spec, hashPartitionComputerFactory, sortFields,
                 comparatorFactories), sorter, 0, writer, 0);
@@ -358,7 +367,7 @@
         typeTraits[1] = new TypeTraits(false);
         BTreeSearchOperatorDescriptor scanner = new BTreeSearchOperatorDescriptor(spec, recordDescriptor,
                 storageManagerInterface, lcManagerProvider, fileSplitProvider, typeTraits, comparatorFactories, null,
-                null, null, true, true, new BTreeDataflowHelperFactory(), false, NoOpOperationCallbackFactory.INSTANCE);
+                null, null, true, true, getIndexDataflowHelperFactory(), false, NoOpOperationCallbackFactory.INSTANCE);
         ClusterConfig.setLocationConstraint(spec, scanner);
 
         /**
@@ -381,8 +390,7 @@
          */
         int[] sortFields = new int[1];
         sortFields[0] = 0;
-        ITuplePartitionComputerFactory hashPartitionComputerFactory = new VertexIdPartitionComputerFactory(
-                new WritableSerializerDeserializerFactory(vertexIdClass));
+        ITuplePartitionComputerFactory hashPartitionComputerFactory = getVertexPartitionComputerFactory();
         spec.connect(new OneToOneConnectorDescriptor(spec), emptyTupleSource, 0, scanner, 0);
         spec.connect(new MToNPartitioningMergingConnectorDescriptor(spec, hashPartitionComputerFactory, sortFields,
                 comparatorFactories), scanner, 0, writer, 0);
@@ -428,7 +436,7 @@
 
         BTreeSearchOperatorDescriptor scanner = new BTreeSearchOperatorDescriptor(spec, recordDescriptor,
                 storageManagerInterface, lcManagerProvider, fileSplitProvider, typeTraits, comparatorFactories, null,
-                null, null, true, true, new BTreeDataflowHelperFactory(), false, NoOpOperationCallbackFactory.INSTANCE);
+                null, null, true, true, getIndexDataflowHelperFactory(), false, NoOpOperationCallbackFactory.INSTANCE);
         ClusterConfig.setLocationConstraint(spec, scanner);
 
         /**
@@ -459,7 +467,7 @@
 
         IFileSplitProvider fileSplitProvider = ClusterConfig.getFileSplitProvider(jobId, indexName);
         IndexDropOperatorDescriptor drop = new IndexDropOperatorDescriptor(spec, storageManagerInterface,
-                lcManagerProvider, fileSplitProvider, new BTreeDataflowHelperFactory());
+                lcManagerProvider, fileSplitProvider, getIndexDataflowHelperFactory());
 
         ClusterConfig.setLocationConstraint(spec, drop);
         spec.addRoot(drop);
@@ -467,6 +475,28 @@
         return spec;
     }
 
+    @SuppressWarnings({ "unchecked", "rawtypes" })
+    protected ITuplePartitionComputerFactory getVertexPartitionComputerFactory() {
+        IConfigurationFactory confFactory = new ConfigurationFactory(conf);
+        Class<? extends VertexPartitioner> partitionerClazz = BspUtils.getVertexPartitionerClass(conf);
+        if (partitionerClazz != null) {
+            return new VertexPartitionComputerFactory(confFactory);
+        } else {
+            return new VertexIdPartitionComputerFactory(new WritableSerializerDeserializerFactory(
+                    BspUtils.getVertexIndexClass(conf)));
+        }
+    }
+
+    protected IIndexDataflowHelperFactory getIndexDataflowHelperFactory() {
+        if (BspUtils.useLSM(conf)) {
+            return new LSMBTreeDataflowHelperFactory(new VirtualBufferCacheProvider(), new ConstantMergePolicyProvider(
+                    3), NoOpOperationTrackerProvider.INSTANCE, SynchronousSchedulerProvider.INSTANCE,
+                    NoOpIOOperationCallback.INSTANCE, 0.01);
+        } else {
+            return new BTreeDataflowHelperFactory();
+       }
+    }
+
     /** generate non-first iteration job */
     protected abstract JobSpecification generateNonFirstIteration(int iteration) throws HyracksException;
 
@@ -476,4 +506,4 @@
     /** generate clean-up job */
     public abstract JobSpecification[] generateCleanup() throws HyracksException;
 
-}
\ No newline at end of file
+}
diff --git a/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGenInnerJoin.java b/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGenInnerJoin.java
index d9a24bc..db6c2c8 100644
--- a/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGenInnerJoin.java
+++ b/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGenInnerJoin.java
@@ -35,14 +35,9 @@
 import edu.uci.ics.hyracks.dataflow.std.group.IAggregatorDescriptorFactory;
 import edu.uci.ics.hyracks.dataflow.std.group.preclustered.PreclusteredGroupOperatorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.sort.ExternalSortOperatorDescriptor;
-import edu.uci.ics.hyracks.storage.am.btree.dataflow.BTreeDataflowHelperFactory;
-import edu.uci.ics.hyracks.storage.am.btree.frames.BTreeNSMInteriorFrameFactory;
-import edu.uci.ics.hyracks.storage.am.btree.frames.BTreeNSMLeafFrameFactory;
-import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
 import edu.uci.ics.hyracks.storage.am.common.dataflow.TreeIndexInsertUpdateDeleteOperatorDescriptor;
 import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallbackFactory;
 import edu.uci.ics.hyracks.storage.am.common.ophelpers.IndexOperation;
-import edu.uci.ics.hyracks.storage.am.common.tuples.TypeAwareTupleWriterFactory;
 import edu.uci.ics.pregelix.api.graph.MsgList;
 import edu.uci.ics.pregelix.api.job.PregelixJob;
 import edu.uci.ics.pregelix.api.util.BspUtils;
@@ -59,11 +54,11 @@
 import edu.uci.ics.pregelix.dataflow.MaterializingWriteOperatorDescriptor;
 import edu.uci.ics.pregelix.dataflow.TerminationStateWriterOperatorDescriptor;
 import edu.uci.ics.pregelix.dataflow.base.IConfigurationFactory;
-import edu.uci.ics.pregelix.dataflow.std.BTreeSearchFunctionUpdateOperatorDescriptor;
 import edu.uci.ics.pregelix.dataflow.std.IndexNestedLoopJoinFunctionUpdateOperatorDescriptor;
 import edu.uci.ics.pregelix.dataflow.std.IndexNestedLoopJoinOperatorDescriptor;
 import edu.uci.ics.pregelix.dataflow.std.RuntimeHookOperatorDescriptor;
 import edu.uci.ics.pregelix.dataflow.std.TreeIndexBulkReLoadOperatorDescriptor;
+import edu.uci.ics.pregelix.dataflow.std.TreeSearchFunctionUpdateOperatorDescriptor;
 import edu.uci.ics.pregelix.dataflow.std.base.IRecordDescriptorFactory;
 import edu.uci.ics.pregelix.dataflow.std.base.IRuntimeHookFactory;
 import edu.uci.ics.pregelix.runtime.function.ComputeUpdateFunctionFactory;
@@ -72,8 +67,6 @@
 import edu.uci.ics.pregelix.runtime.touchpoint.PostSuperStepRuntimeHookFactory;
 import edu.uci.ics.pregelix.runtime.touchpoint.PreSuperStepRuntimeHookFactory;
 import edu.uci.ics.pregelix.runtime.touchpoint.RuntimeHookFactory;
-import edu.uci.ics.pregelix.runtime.touchpoint.VertexIdPartitionComputerFactory;
-import edu.uci.ics.pregelix.runtime.touchpoint.WritableSerializerDeserializerFactory;
 
 public class JobGenInnerJoin extends JobGen {
 
@@ -135,12 +128,11 @@
                 vertexClass.getName());
         RecordDescriptor rdDelete = DataflowUtils.getRecordDescriptorFromWritableClasses(vertexIdClass.getName());
 
-        BTreeSearchFunctionUpdateOperatorDescriptor scanner = new BTreeSearchFunctionUpdateOperatorDescriptor(spec,
+        TreeSearchFunctionUpdateOperatorDescriptor scanner = new TreeSearchFunctionUpdateOperatorDescriptor(spec,
                 recordDescriptor, storageManagerInterface, lcManagerProvider, fileSplitProvider, typeTraits,
                 comparatorFactories, JobGenUtil.getForwardScan(iteration), null, null, true, true,
-                new BTreeDataflowHelperFactory(), inputRdFactory, 6,
-                new StartComputeUpdateFunctionFactory(confFactory), preHookFactory, null, rdUnnestedMessage, rdDummy,
-                rdPartialAggregate, rdInsert, rdDelete, rdFinal);
+                getIndexDataflowHelperFactory(), inputRdFactory, 6, new StartComputeUpdateFunctionFactory(confFactory),
+                preHookFactory, null, rdUnnestedMessage, rdDummy, rdPartialAggregate, rdInsert, rdDelete, rdFinal);
         ClusterConfig.setLocationConstraint(spec, scanner);
 
         /**
@@ -163,18 +155,18 @@
          * construct bulk-load index operator
          */
         int[] fieldPermutation = new int[] { 0, 1 };
+        int[] keyFields = new int[] { 0 };
         IBinaryComparatorFactory[] indexCmpFactories = new IBinaryComparatorFactory[1];
         indexCmpFactories[0] = JobGenUtil.getIBinaryComparatorFactory(iteration + 1,
                 WritableComparator.get(vertexIdClass).getClass());
         TreeIndexBulkReLoadOperatorDescriptor btreeBulkLoad = new TreeIndexBulkReLoadOperatorDescriptor(spec,
                 storageManagerInterface, lcManagerProvider, secondaryFileSplitProvider, typeTraits, indexCmpFactories,
-                fieldPermutation, DEFAULT_BTREE_FILL_FACTOR, new BTreeDataflowHelperFactory());
+                fieldPermutation, keyFields, DEFAULT_BTREE_FILL_FACTOR, getIndexDataflowHelperFactory());
         ClusterConfig.setLocationConstraint(spec, btreeBulkLoad);
 
         /**
          * construct local sort operator
          */
-        int[] keyFields = new int[] { 0 };
         INormalizedKeyComputerFactory nkmFactory = JobGenUtil.getINormalizedKeyComputerFactory(conf);
         IBinaryComparatorFactory[] sortCmpFactories = new IBinaryComparatorFactory[1];
         sortCmpFactories[0] = JobGenUtil.getIBinaryComparatorFactory(iteration, WritableComparator.get(vertexIdClass)
@@ -223,7 +215,7 @@
          */
         TreeIndexInsertUpdateDeleteOperatorDescriptor insertOp = new TreeIndexInsertUpdateDeleteOperatorDescriptor(
                 spec, rdInsert, storageManagerInterface, lcManagerProvider, fileSplitProvider, typeTraits,
-                comparatorFactories, null, fieldPermutation, IndexOperation.INSERT, new BTreeDataflowHelperFactory(),
+                comparatorFactories, null, fieldPermutation, IndexOperation.INSERT, getIndexDataflowHelperFactory(),
                 null, NoOpOperationCallbackFactory.INSTANCE);
         ClusterConfig.setLocationConstraint(spec, insertOp);
 
@@ -234,7 +226,7 @@
         TreeIndexInsertUpdateDeleteOperatorDescriptor deleteOp = new TreeIndexInsertUpdateDeleteOperatorDescriptor(
                 spec, rdDelete, storageManagerInterface, lcManagerProvider, fileSplitProvider, typeTraits,
                 comparatorFactories, null, fieldPermutationDelete, IndexOperation.DELETE,
-                new BTreeDataflowHelperFactory(), null, NoOpOperationCallbackFactory.INSTANCE);
+                getIndexDataflowHelperFactory(), null, NoOpOperationCallbackFactory.INSTANCE);
         ClusterConfig.setLocationConstraint(spec, deleteOp);
 
         /** construct empty sink operator */
@@ -245,8 +237,7 @@
         EmptySinkOperatorDescriptor emptySink4 = new EmptySinkOperatorDescriptor(spec);
         ClusterConfig.setLocationConstraint(spec, emptySink4);
 
-        ITuplePartitionComputerFactory partionFactory = new VertexIdPartitionComputerFactory(
-                new WritableSerializerDeserializerFactory(vertexIdClass));
+        ITuplePartitionComputerFactory partionFactory = getVertexPartitionComputerFactory();
         ITuplePartitionComputerFactory unifyingPartitionComputerFactory = new MergePartitionComputerFactory();
 
         /** connect all operators **/
@@ -339,14 +330,9 @@
         ITypeTraits[] typeTraits = new ITypeTraits[2];
         typeTraits[0] = new TypeTraits(false);
         typeTraits[1] = new TypeTraits(false);
-        ITreeIndexFrameFactory interiorFrameFactory = new BTreeNSMInteriorFrameFactory(new TypeAwareTupleWriterFactory(
-                typeTraits));
-        ITreeIndexFrameFactory leafFrameFactory = new BTreeNSMLeafFrameFactory(new TypeAwareTupleWriterFactory(
-                typeTraits));
         IndexNestedLoopJoinOperatorDescriptor setUnion = new IndexNestedLoopJoinOperatorDescriptor(spec, rdFinal,
-                storageManagerInterface, lcManagerProvider, secondaryFileSplitProviderRead, interiorFrameFactory,
-                leafFrameFactory, typeTraits, comparatorFactories, true, keyFields, keyFields, true, true,
-                new BTreeDataflowHelperFactory(), true);
+                storageManagerInterface, lcManagerProvider, secondaryFileSplitProviderRead, typeTraits,
+                comparatorFactories, true, keyFields, keyFields, true, true, getIndexDataflowHelperFactory(), true);
         ClusterConfig.setLocationConstraint(spec, setUnion);
 
         /**
@@ -364,7 +350,7 @@
         IndexNestedLoopJoinFunctionUpdateOperatorDescriptor join = new IndexNestedLoopJoinFunctionUpdateOperatorDescriptor(
                 spec, storageManagerInterface, lcManagerProvider, fileSplitProvider, typeTraits, comparatorFactories,
                 JobGenUtil.getForwardScan(iteration), keyFields, keyFields, true, true,
-                new BTreeDataflowHelperFactory(), inputRdFactory, 6, new ComputeUpdateFunctionFactory(confFactory),
+                getIndexDataflowHelperFactory(), inputRdFactory, 6, new ComputeUpdateFunctionFactory(confFactory),
                 preHookFactory, null, rdUnnestedMessage, rdDummy, rdPartialAggregate, rdInsert, rdDelete, rdFinal);
         ClusterConfig.setLocationConstraint(spec, join);
 
@@ -379,7 +365,8 @@
         IFileSplitProvider secondaryFileSplitProviderWrite = ClusterConfig.getFileSplitProvider(jobId, writeFile);
         TreeIndexBulkReLoadOperatorDescriptor btreeBulkLoad = new TreeIndexBulkReLoadOperatorDescriptor(spec,
                 storageManagerInterface, lcManagerProvider, secondaryFileSplitProviderWrite, typeTraits,
-                indexCmpFactories, fieldPermutation, DEFAULT_BTREE_FILL_FACTOR, new BTreeDataflowHelperFactory());
+                indexCmpFactories, fieldPermutation, keyFields, DEFAULT_BTREE_FILL_FACTOR,
+                getIndexDataflowHelperFactory());
         ClusterConfig.setLocationConstraint(spec, btreeBulkLoad);
 
         /**
@@ -447,7 +434,7 @@
          */
         TreeIndexInsertUpdateDeleteOperatorDescriptor insertOp = new TreeIndexInsertUpdateDeleteOperatorDescriptor(
                 spec, rdInsert, storageManagerInterface, lcManagerProvider, fileSplitProvider, typeTraits,
-                comparatorFactories, null, fieldPermutation, IndexOperation.INSERT, new BTreeDataflowHelperFactory(),
+                comparatorFactories, null, fieldPermutation, IndexOperation.INSERT, getIndexDataflowHelperFactory(),
                 null, NoOpOperationCallbackFactory.INSTANCE);
         ClusterConfig.setLocationConstraint(spec, insertOp);
 
@@ -458,7 +445,7 @@
         TreeIndexInsertUpdateDeleteOperatorDescriptor deleteOp = new TreeIndexInsertUpdateDeleteOperatorDescriptor(
                 spec, rdDelete, storageManagerInterface, lcManagerProvider, fileSplitProvider, typeTraits,
                 comparatorFactories, null, fieldPermutationDelete, IndexOperation.DELETE,
-                new BTreeDataflowHelperFactory(), null, NoOpOperationCallbackFactory.INSTANCE);
+                getIndexDataflowHelperFactory(), null, NoOpOperationCallbackFactory.INSTANCE);
         ClusterConfig.setLocationConstraint(spec, deleteOp);
 
         /** construct empty sink operator */
@@ -470,8 +457,7 @@
         ClusterConfig.setLocationConstraint(spec, emptySink4);
 
         ITuplePartitionComputerFactory unifyingPartitionComputerFactory = new MergePartitionComputerFactory();
-        ITuplePartitionComputerFactory partionFactory = new VertexIdPartitionComputerFactory(
-                new WritableSerializerDeserializerFactory(vertexIdClass));
+        ITuplePartitionComputerFactory partionFactory = getVertexPartitionComputerFactory();
         /** connect all operators **/
         spec.connect(new OneToOneConnectorDescriptor(spec), emptyTupleSource, 0, preSuperStep, 0);
         spec.connect(new OneToOneConnectorDescriptor(spec), preSuperStep, 0, materializeRead, 0);
diff --git a/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGenOuterJoin.java b/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGenOuterJoin.java
index 5faf122..3af8921 100644
--- a/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGenOuterJoin.java
+++ b/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGenOuterJoin.java
@@ -35,14 +35,9 @@
 import edu.uci.ics.hyracks.dataflow.std.group.IAggregatorDescriptorFactory;
 import edu.uci.ics.hyracks.dataflow.std.group.preclustered.PreclusteredGroupOperatorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.sort.ExternalSortOperatorDescriptor;
-import edu.uci.ics.hyracks.storage.am.btree.dataflow.BTreeDataflowHelperFactory;
-import edu.uci.ics.hyracks.storage.am.btree.frames.BTreeNSMInteriorFrameFactory;
-import edu.uci.ics.hyracks.storage.am.btree.frames.BTreeNSMLeafFrameFactory;
-import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
 import edu.uci.ics.hyracks.storage.am.common.dataflow.TreeIndexInsertUpdateDeleteOperatorDescriptor;
 import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallbackFactory;
 import edu.uci.ics.hyracks.storage.am.common.ophelpers.IndexOperation;
-import edu.uci.ics.hyracks.storage.am.common.tuples.TypeAwareTupleWriterFactory;
 import edu.uci.ics.pregelix.api.graph.MsgList;
 import edu.uci.ics.pregelix.api.job.PregelixJob;
 import edu.uci.ics.pregelix.api.util.BspUtils;
@@ -59,9 +54,9 @@
 import edu.uci.ics.pregelix.dataflow.MaterializingWriteOperatorDescriptor;
 import edu.uci.ics.pregelix.dataflow.TerminationStateWriterOperatorDescriptor;
 import edu.uci.ics.pregelix.dataflow.base.IConfigurationFactory;
-import edu.uci.ics.pregelix.dataflow.std.BTreeSearchFunctionUpdateOperatorDescriptor;
 import edu.uci.ics.pregelix.dataflow.std.IndexNestedLoopJoinFunctionUpdateOperatorDescriptor;
 import edu.uci.ics.pregelix.dataflow.std.RuntimeHookOperatorDescriptor;
+import edu.uci.ics.pregelix.dataflow.std.TreeSearchFunctionUpdateOperatorDescriptor;
 import edu.uci.ics.pregelix.dataflow.std.base.IRecordDescriptorFactory;
 import edu.uci.ics.pregelix.dataflow.std.base.IRuntimeHookFactory;
 import edu.uci.ics.pregelix.runtime.function.ComputeUpdateFunctionFactory;
@@ -72,8 +67,6 @@
 import edu.uci.ics.pregelix.runtime.touchpoint.PreSuperStepRuntimeHookFactory;
 import edu.uci.ics.pregelix.runtime.touchpoint.RuntimeHookFactory;
 import edu.uci.ics.pregelix.runtime.touchpoint.VertexIdNullWriterFactory;
-import edu.uci.ics.pregelix.runtime.touchpoint.VertexIdPartitionComputerFactory;
-import edu.uci.ics.pregelix.runtime.touchpoint.WritableSerializerDeserializerFactory;
 
 public class JobGenOuterJoin extends JobGen {
 
@@ -129,12 +122,11 @@
                 vertexClass.getName());
         RecordDescriptor rdDelete = DataflowUtils.getRecordDescriptorFromWritableClasses(vertexIdClass.getName());
 
-        BTreeSearchFunctionUpdateOperatorDescriptor scanner = new BTreeSearchFunctionUpdateOperatorDescriptor(spec,
+        TreeSearchFunctionUpdateOperatorDescriptor scanner = new TreeSearchFunctionUpdateOperatorDescriptor(spec,
                 recordDescriptor, storageManagerInterface, lcManagerProvider, fileSplitProvider, typeTraits,
                 comparatorFactories, JobGenUtil.getForwardScan(iteration), null, null, true, true,
-                new BTreeDataflowHelperFactory(), inputRdFactory, 5,
-                new StartComputeUpdateFunctionFactory(confFactory), preHookFactory, null, rdUnnestedMessage, rdDummy,
-                rdPartialAggregate, rdInsert, rdDelete);
+                getIndexDataflowHelperFactory(), inputRdFactory, 5, new StartComputeUpdateFunctionFactory(confFactory),
+                preHookFactory, null, rdUnnestedMessage, rdDummy, rdPartialAggregate, rdInsert, rdDelete);
         ClusterConfig.setLocationConstraint(spec, scanner);
 
         /**
@@ -206,8 +198,8 @@
         int[] fieldPermutation = new int[] { 0, 1 };
         TreeIndexInsertUpdateDeleteOperatorDescriptor insertOp = new TreeIndexInsertUpdateDeleteOperatorDescriptor(
                 spec, rdInsert, storageManagerInterface, lcManagerProvider, fileSplitProvider, typeTraits,
-                comparatorFactories, null, fieldPermutation, IndexOperation.INSERT, new BTreeDataflowHelperFactory(),
-                null, NoOpOperationCallbackFactory.INSTANCE);
+                comparatorFactories, null, fieldPermutation, IndexOperation.INSERT, getIndexDataflowHelperFactory(), null,
+                NoOpOperationCallbackFactory.INSTANCE);
         ClusterConfig.setLocationConstraint(spec, insertOp);
 
         /**
@@ -216,8 +208,8 @@
         int[] fieldPermutationDelete = new int[] { 0 };
         TreeIndexInsertUpdateDeleteOperatorDescriptor deleteOp = new TreeIndexInsertUpdateDeleteOperatorDescriptor(
                 spec, rdDelete, storageManagerInterface, lcManagerProvider, fileSplitProvider, typeTraits,
-                comparatorFactories, null, fieldPermutationDelete, IndexOperation.DELETE,
-                new BTreeDataflowHelperFactory(), null, NoOpOperationCallbackFactory.INSTANCE);
+                comparatorFactories, null, fieldPermutationDelete, IndexOperation.DELETE, getIndexDataflowHelperFactory(),
+                null, NoOpOperationCallbackFactory.INSTANCE);
         ClusterConfig.setLocationConstraint(spec, deleteOp);
 
         /** construct empty sink operator */
@@ -227,8 +219,7 @@
         EmptySinkOperatorDescriptor emptySink4 = new EmptySinkOperatorDescriptor(spec);
         ClusterConfig.setLocationConstraint(spec, emptySink4);
 
-        ITuplePartitionComputerFactory partionFactory = new VertexIdPartitionComputerFactory(
-                new WritableSerializerDeserializerFactory(vertexIdClass));
+        ITuplePartitionComputerFactory partionFactory = getVertexPartitionComputerFactory();
         /** connect all operators **/
         spec.connect(new OneToOneConnectorDescriptor(spec), emptyTupleSource, 0, preSuperStep, 0);
         spec.connect(new OneToOneConnectorDescriptor(spec), preSuperStep, 0, scanner, 0);
@@ -318,10 +309,6 @@
         ITypeTraits[] typeTraits = new ITypeTraits[2];
         typeTraits[0] = new TypeTraits(false);
         typeTraits[1] = new TypeTraits(false);
-        ITreeIndexFrameFactory interiorFrameFactory = new BTreeNSMInteriorFrameFactory(new TypeAwareTupleWriterFactory(
-                typeTraits));
-        ITreeIndexFrameFactory leafFrameFactory = new BTreeNSMLeafFrameFactory(new TypeAwareTupleWriterFactory(
-                typeTraits));
         INullWriterFactory[] nullWriterFactories = new INullWriterFactory[2];
         nullWriterFactories[0] = VertexIdNullWriterFactory.INSTANCE;
         nullWriterFactories[1] = MsgListNullWriterFactory.INSTANCE;
@@ -335,11 +322,10 @@
                 vertexIdClass.getName(), MsgList.class.getName(), vertexIdClass.getName(), vertexClass.getName());
 
         IndexNestedLoopJoinFunctionUpdateOperatorDescriptor join = new IndexNestedLoopJoinFunctionUpdateOperatorDescriptor(
-                spec, storageManagerInterface, lcManagerProvider, fileSplitProvider, interiorFrameFactory,
-                leafFrameFactory, typeTraits, comparatorFactories, JobGenUtil.getForwardScan(iteration), keyFields,
-                keyFields, true, true, new BTreeDataflowHelperFactory(), true, nullWriterFactories, inputRdFactory, 5,
-                new ComputeUpdateFunctionFactory(confFactory), preHookFactory, null, rdUnnestedMessage, rdDummy,
-                rdPartialAggregate, rdInsert, rdDelete);
+                spec, storageManagerInterface, lcManagerProvider, fileSplitProvider, typeTraits, comparatorFactories,
+                JobGenUtil.getForwardScan(iteration), keyFields, keyFields, true, true, getIndexDataflowHelperFactory(), true,
+                nullWriterFactories, inputRdFactory, 5, new ComputeUpdateFunctionFactory(confFactory), preHookFactory,
+                null, rdUnnestedMessage, rdDummy, rdPartialAggregate, rdInsert, rdDelete);
         ClusterConfig.setLocationConstraint(spec, join);
 
         /**
@@ -408,8 +394,8 @@
         int[] fieldPermutation = new int[] { 0, 1 };
         TreeIndexInsertUpdateDeleteOperatorDescriptor insertOp = new TreeIndexInsertUpdateDeleteOperatorDescriptor(
                 spec, rdInsert, storageManagerInterface, lcManagerProvider, fileSplitProvider, typeTraits,
-                comparatorFactories, null, fieldPermutation, IndexOperation.INSERT, new BTreeDataflowHelperFactory(),
-                null, NoOpOperationCallbackFactory.INSTANCE);
+                comparatorFactories, null, fieldPermutation, IndexOperation.INSERT, getIndexDataflowHelperFactory(), null,
+                NoOpOperationCallbackFactory.INSTANCE);
         ClusterConfig.setLocationConstraint(spec, insertOp);
 
         /**
@@ -418,7 +404,7 @@
         int[] fieldPermutationDelete = new int[] { 0 };
         TreeIndexInsertUpdateDeleteOperatorDescriptor deleteOp = new TreeIndexInsertUpdateDeleteOperatorDescriptor(
                 spec, rdDelete, storageManagerInterface, lcManagerProvider, fileSplitProvider, typeTraits,
-                comparatorFactories, null, fieldPermutationDelete, IndexOperation.DELETE, new BTreeDataflowHelperFactory(),
+                comparatorFactories, null, fieldPermutationDelete, IndexOperation.DELETE, getIndexDataflowHelperFactory(),
                 null, NoOpOperationCallbackFactory.INSTANCE);
         ClusterConfig.setLocationConstraint(spec, deleteOp);
 
@@ -431,8 +417,7 @@
         ClusterConfig.setLocationConstraint(spec, emptySink4);
 
         ITuplePartitionComputerFactory unifyingPartitionComputerFactory = new MergePartitionComputerFactory();
-        ITuplePartitionComputerFactory partionFactory = new VertexIdPartitionComputerFactory(
-                new WritableSerializerDeserializerFactory(vertexIdClass));
+        ITuplePartitionComputerFactory partionFactory = getVertexPartitionComputerFactory();
 
         /** connect all operators **/
         spec.connect(new OneToOneConnectorDescriptor(spec), emptyTupleSource, 0, preSuperStep, 0);
diff --git a/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGenOuterJoinSingleSort.java b/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGenOuterJoinSingleSort.java
index fee738e..50949aa 100644
--- a/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGenOuterJoinSingleSort.java
+++ b/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGenOuterJoinSingleSort.java
@@ -34,14 +34,9 @@
 import edu.uci.ics.hyracks.dataflow.std.group.IAggregatorDescriptorFactory;
 import edu.uci.ics.hyracks.dataflow.std.group.preclustered.PreclusteredGroupOperatorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.sort.ExternalSortOperatorDescriptor;
-import edu.uci.ics.hyracks.storage.am.btree.dataflow.BTreeDataflowHelperFactory;
-import edu.uci.ics.hyracks.storage.am.btree.frames.BTreeNSMInteriorFrameFactory;
-import edu.uci.ics.hyracks.storage.am.btree.frames.BTreeNSMLeafFrameFactory;
-import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
 import edu.uci.ics.hyracks.storage.am.common.dataflow.TreeIndexInsertUpdateDeleteOperatorDescriptor;
 import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallbackFactory;
 import edu.uci.ics.hyracks.storage.am.common.ophelpers.IndexOperation;
-import edu.uci.ics.hyracks.storage.am.common.tuples.TypeAwareTupleWriterFactory;
 import edu.uci.ics.pregelix.api.graph.MsgList;
 import edu.uci.ics.pregelix.api.job.PregelixJob;
 import edu.uci.ics.pregelix.api.util.BspUtils;
@@ -58,9 +53,9 @@
 import edu.uci.ics.pregelix.dataflow.MaterializingWriteOperatorDescriptor;
 import edu.uci.ics.pregelix.dataflow.TerminationStateWriterOperatorDescriptor;
 import edu.uci.ics.pregelix.dataflow.base.IConfigurationFactory;
-import edu.uci.ics.pregelix.dataflow.std.BTreeSearchFunctionUpdateOperatorDescriptor;
 import edu.uci.ics.pregelix.dataflow.std.IndexNestedLoopJoinFunctionUpdateOperatorDescriptor;
 import edu.uci.ics.pregelix.dataflow.std.RuntimeHookOperatorDescriptor;
+import edu.uci.ics.pregelix.dataflow.std.TreeSearchFunctionUpdateOperatorDescriptor;
 import edu.uci.ics.pregelix.dataflow.std.base.IRecordDescriptorFactory;
 import edu.uci.ics.pregelix.dataflow.std.base.IRuntimeHookFactory;
 import edu.uci.ics.pregelix.runtime.function.ComputeUpdateFunctionFactory;
@@ -71,8 +66,6 @@
 import edu.uci.ics.pregelix.runtime.touchpoint.PreSuperStepRuntimeHookFactory;
 import edu.uci.ics.pregelix.runtime.touchpoint.RuntimeHookFactory;
 import edu.uci.ics.pregelix.runtime.touchpoint.VertexIdNullWriterFactory;
-import edu.uci.ics.pregelix.runtime.touchpoint.VertexIdPartitionComputerFactory;
-import edu.uci.ics.pregelix.runtime.touchpoint.WritableSerializerDeserializerFactory;
 
 public class JobGenOuterJoinSingleSort extends JobGen {
 
@@ -131,12 +124,11 @@
                 vertexClass.getName());
         RecordDescriptor rdDelete = DataflowUtils.getRecordDescriptorFromWritableClasses(vertexIdClass.getName());
 
-        BTreeSearchFunctionUpdateOperatorDescriptor scanner = new BTreeSearchFunctionUpdateOperatorDescriptor(spec,
+        TreeSearchFunctionUpdateOperatorDescriptor scanner = new TreeSearchFunctionUpdateOperatorDescriptor(spec,
                 recordDescriptor, storageManagerInterface, lcManagerProvider, fileSplitProvider, typeTraits,
                 comparatorFactories, JobGenUtil.getForwardScan(iteration), null, null, true, true,
-                new BTreeDataflowHelperFactory(), inputRdFactory, 5,
-                new StartComputeUpdateFunctionFactory(confFactory), preHookFactory, null, rdUnnestedMessage, rdDummy,
-                rdPartialAggregate, rdInsert, rdDelete);
+                getIndexDataflowHelperFactory(), inputRdFactory, 5, new StartComputeUpdateFunctionFactory(confFactory),
+                preHookFactory, null, rdUnnestedMessage, rdDummy, rdPartialAggregate, rdInsert, rdDelete);
         ClusterConfig.setLocationConstraint(spec, scanner);
 
         /**
@@ -198,8 +190,8 @@
         int[] fieldPermutation = new int[] { 0, 1 };
         TreeIndexInsertUpdateDeleteOperatorDescriptor insertOp = new TreeIndexInsertUpdateDeleteOperatorDescriptor(
                 spec, rdInsert, storageManagerInterface, lcManagerProvider, fileSplitProvider, typeTraits,
-                comparatorFactories, null, fieldPermutation, IndexOperation.INSERT, new BTreeDataflowHelperFactory(),
-                null, NoOpOperationCallbackFactory.INSTANCE);
+                comparatorFactories, null, fieldPermutation, IndexOperation.INSERT, getIndexDataflowHelperFactory(), null,
+                NoOpOperationCallbackFactory.INSTANCE);
         ClusterConfig.setLocationConstraint(spec, insertOp);
 
         /**
@@ -208,8 +200,8 @@
         int[] fieldPermutationDelete = new int[] { 0 };
         TreeIndexInsertUpdateDeleteOperatorDescriptor deleteOp = new TreeIndexInsertUpdateDeleteOperatorDescriptor(
                 spec, rdDelete, storageManagerInterface, lcManagerProvider, fileSplitProvider, typeTraits,
-                comparatorFactories, null, fieldPermutationDelete, IndexOperation.DELETE,
-                new BTreeDataflowHelperFactory(), null, NoOpOperationCallbackFactory.INSTANCE);
+                comparatorFactories, null, fieldPermutationDelete, IndexOperation.DELETE, getIndexDataflowHelperFactory(),
+                null, NoOpOperationCallbackFactory.INSTANCE);
         ClusterConfig.setLocationConstraint(spec, deleteOp);
 
         /** construct empty sink operator */
@@ -221,8 +213,7 @@
         ClusterConfig.setLocationConstraint(spec, emptySink4);
 
         ITuplePartitionComputerFactory unifyingPartitionComputerFactory = new MergePartitionComputerFactory();
-        ITuplePartitionComputerFactory partionFactory = new VertexIdPartitionComputerFactory(
-                new WritableSerializerDeserializerFactory(vertexIdClass));
+        ITuplePartitionComputerFactory partionFactory = getVertexPartitionComputerFactory();
         /** connect all operators **/
         spec.connect(new OneToOneConnectorDescriptor(spec), emptyTupleSource, 0, preSuperStep, 0);
         spec.connect(new OneToOneConnectorDescriptor(spec), preSuperStep, 0, scanner, 0);
@@ -307,10 +298,6 @@
         ITypeTraits[] typeTraits = new ITypeTraits[2];
         typeTraits[0] = new TypeTraits(false);
         typeTraits[1] = new TypeTraits(false);
-        ITreeIndexFrameFactory interiorFrameFactory = new BTreeNSMInteriorFrameFactory(new TypeAwareTupleWriterFactory(
-                typeTraits));
-        ITreeIndexFrameFactory leafFrameFactory = new BTreeNSMLeafFrameFactory(new TypeAwareTupleWriterFactory(
-                typeTraits));
         INullWriterFactory[] nullWriterFactories = new INullWriterFactory[2];
         nullWriterFactories[0] = VertexIdNullWriterFactory.INSTANCE;
         nullWriterFactories[1] = MsgListNullWriterFactory.INSTANCE;
@@ -324,11 +311,10 @@
                 vertexIdClass.getName(), MsgList.class.getName(), vertexIdClass.getName(), vertexClass.getName());
 
         IndexNestedLoopJoinFunctionUpdateOperatorDescriptor join = new IndexNestedLoopJoinFunctionUpdateOperatorDescriptor(
-                spec, storageManagerInterface, lcManagerProvider, fileSplitProvider, interiorFrameFactory,
-                leafFrameFactory, typeTraits, comparatorFactories, JobGenUtil.getForwardScan(iteration), keyFields,
-                keyFields, true, true, new BTreeDataflowHelperFactory(), true, nullWriterFactories, inputRdFactory, 5,
-                new ComputeUpdateFunctionFactory(confFactory), preHookFactory, null, rdUnnestedMessage, rdDummy,
-                rdPartialAggregate, rdInsert, rdDelete);
+                spec, storageManagerInterface, lcManagerProvider, fileSplitProvider, typeTraits, comparatorFactories,
+                JobGenUtil.getForwardScan(iteration), keyFields, keyFields, true, true, getIndexDataflowHelperFactory(), true,
+                nullWriterFactories, inputRdFactory, 5, new ComputeUpdateFunctionFactory(confFactory), preHookFactory,
+                null, rdUnnestedMessage, rdDummy, rdPartialAggregate, rdInsert, rdDelete);
         ClusterConfig.setLocationConstraint(spec, join);
 
         /**
@@ -385,8 +371,8 @@
         int[] fieldPermutation = new int[] { 0, 1 };
         TreeIndexInsertUpdateDeleteOperatorDescriptor insertOp = new TreeIndexInsertUpdateDeleteOperatorDescriptor(
                 spec, rdInsert, storageManagerInterface, lcManagerProvider, fileSplitProvider, typeTraits,
-                comparatorFactories, null, fieldPermutation, IndexOperation.INSERT, new BTreeDataflowHelperFactory(),
-                null, NoOpOperationCallbackFactory.INSTANCE);
+                comparatorFactories, null, fieldPermutation, IndexOperation.INSERT, getIndexDataflowHelperFactory(), null,
+                NoOpOperationCallbackFactory.INSTANCE);
         ClusterConfig.setLocationConstraint(spec, insertOp);
 
         /**
@@ -395,8 +381,8 @@
         int[] fieldPermutationDelete = new int[] { 0 };
         TreeIndexInsertUpdateDeleteOperatorDescriptor deleteOp = new TreeIndexInsertUpdateDeleteOperatorDescriptor(
                 spec, rdDelete, storageManagerInterface, lcManagerProvider, fileSplitProvider, typeTraits,
-                comparatorFactories, null, fieldPermutationDelete, IndexOperation.DELETE,
-                new BTreeDataflowHelperFactory(), null, NoOpOperationCallbackFactory.INSTANCE);
+                comparatorFactories, null, fieldPermutationDelete, IndexOperation.DELETE, getIndexDataflowHelperFactory(),
+                null, NoOpOperationCallbackFactory.INSTANCE);
         ClusterConfig.setLocationConstraint(spec, deleteOp);
 
         /** construct empty sink operator */
@@ -408,8 +394,7 @@
         ClusterConfig.setLocationConstraint(spec, emptySink4);
 
         ITuplePartitionComputerFactory unifyingPartitionComputerFactory = new MergePartitionComputerFactory();
-        ITuplePartitionComputerFactory partionFactory = new VertexIdPartitionComputerFactory(
-                new WritableSerializerDeserializerFactory(vertexIdClass));
+        ITuplePartitionComputerFactory partionFactory = getVertexPartitionComputerFactory();
 
         /** connect all operators **/
         spec.connect(new OneToOneConnectorDescriptor(spec), emptyTupleSource, 0, preSuperStep, 0);
diff --git a/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGenOuterJoinSort.java b/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGenOuterJoinSort.java
index eef6b7e..362e413 100644
--- a/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGenOuterJoinSort.java
+++ b/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGenOuterJoinSort.java
@@ -34,14 +34,9 @@
 import edu.uci.ics.hyracks.dataflow.std.group.IAggregatorDescriptorFactory;
 import edu.uci.ics.hyracks.dataflow.std.group.preclustered.PreclusteredGroupOperatorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.sort.ExternalSortOperatorDescriptor;
-import edu.uci.ics.hyracks.storage.am.btree.dataflow.BTreeDataflowHelperFactory;
-import edu.uci.ics.hyracks.storage.am.btree.frames.BTreeNSMInteriorFrameFactory;
-import edu.uci.ics.hyracks.storage.am.btree.frames.BTreeNSMLeafFrameFactory;
-import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
 import edu.uci.ics.hyracks.storage.am.common.dataflow.TreeIndexInsertUpdateDeleteOperatorDescriptor;
 import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallbackFactory;
 import edu.uci.ics.hyracks.storage.am.common.ophelpers.IndexOperation;
-import edu.uci.ics.hyracks.storage.am.common.tuples.TypeAwareTupleWriterFactory;
 import edu.uci.ics.pregelix.api.graph.MsgList;
 import edu.uci.ics.pregelix.api.job.PregelixJob;
 import edu.uci.ics.pregelix.api.util.BspUtils;
@@ -58,9 +53,9 @@
 import edu.uci.ics.pregelix.dataflow.MaterializingWriteOperatorDescriptor;
 import edu.uci.ics.pregelix.dataflow.TerminationStateWriterOperatorDescriptor;
 import edu.uci.ics.pregelix.dataflow.base.IConfigurationFactory;
-import edu.uci.ics.pregelix.dataflow.std.BTreeSearchFunctionUpdateOperatorDescriptor;
 import edu.uci.ics.pregelix.dataflow.std.IndexNestedLoopJoinFunctionUpdateOperatorDescriptor;
 import edu.uci.ics.pregelix.dataflow.std.RuntimeHookOperatorDescriptor;
+import edu.uci.ics.pregelix.dataflow.std.TreeSearchFunctionUpdateOperatorDescriptor;
 import edu.uci.ics.pregelix.dataflow.std.base.IRecordDescriptorFactory;
 import edu.uci.ics.pregelix.dataflow.std.base.IRuntimeHookFactory;
 import edu.uci.ics.pregelix.runtime.function.ComputeUpdateFunctionFactory;
@@ -71,8 +66,6 @@
 import edu.uci.ics.pregelix.runtime.touchpoint.PreSuperStepRuntimeHookFactory;
 import edu.uci.ics.pregelix.runtime.touchpoint.RuntimeHookFactory;
 import edu.uci.ics.pregelix.runtime.touchpoint.VertexIdNullWriterFactory;
-import edu.uci.ics.pregelix.runtime.touchpoint.VertexIdPartitionComputerFactory;
-import edu.uci.ics.pregelix.runtime.touchpoint.WritableSerializerDeserializerFactory;
 
 public class JobGenOuterJoinSort extends JobGen {
 
@@ -128,12 +121,11 @@
                 vertexClass.getName());
         RecordDescriptor rdDelete = DataflowUtils.getRecordDescriptorFromWritableClasses(vertexIdClass.getName());
 
-        BTreeSearchFunctionUpdateOperatorDescriptor scanner = new BTreeSearchFunctionUpdateOperatorDescriptor(spec,
+        TreeSearchFunctionUpdateOperatorDescriptor scanner = new TreeSearchFunctionUpdateOperatorDescriptor(spec,
                 recordDescriptor, storageManagerInterface, lcManagerProvider, fileSplitProvider, typeTraits,
                 comparatorFactories, JobGenUtil.getForwardScan(iteration), null, null, true, true,
-                new BTreeDataflowHelperFactory(), inputRdFactory, 5,
-                new StartComputeUpdateFunctionFactory(confFactory), preHookFactory, null, rdUnnestedMessage, rdDummy,
-                rdPartialAggregate, rdInsert, rdDelete);
+                getIndexDataflowHelperFactory(), inputRdFactory, 5, new StartComputeUpdateFunctionFactory(confFactory),
+                preHookFactory, null, rdUnnestedMessage, rdDummy, rdPartialAggregate, rdInsert, rdDelete);
         ClusterConfig.setLocationConstraint(spec, scanner);
 
         /**
@@ -212,8 +204,8 @@
         int[] fieldPermutation = new int[] { 0, 1 };
         TreeIndexInsertUpdateDeleteOperatorDescriptor insertOp = new TreeIndexInsertUpdateDeleteOperatorDescriptor(
                 spec, rdInsert, storageManagerInterface, lcManagerProvider, fileSplitProvider, typeTraits,
-                comparatorFactories, null, fieldPermutation, IndexOperation.INSERT, new BTreeDataflowHelperFactory(),
-                null, NoOpOperationCallbackFactory.INSTANCE);
+                comparatorFactories, null, fieldPermutation, IndexOperation.INSERT, getIndexDataflowHelperFactory(), null,
+                NoOpOperationCallbackFactory.INSTANCE);
         ClusterConfig.setLocationConstraint(spec, insertOp);
 
         /**
@@ -222,8 +214,8 @@
         int[] fieldPermutationDelete = new int[] { 0 };
         TreeIndexInsertUpdateDeleteOperatorDescriptor deleteOp = new TreeIndexInsertUpdateDeleteOperatorDescriptor(
                 spec, rdDelete, storageManagerInterface, lcManagerProvider, fileSplitProvider, typeTraits,
-                comparatorFactories, null, fieldPermutationDelete, IndexOperation.DELETE,
-                new BTreeDataflowHelperFactory(), null, NoOpOperationCallbackFactory.INSTANCE);
+                comparatorFactories, null, fieldPermutationDelete, IndexOperation.DELETE, getIndexDataflowHelperFactory(),
+                null, NoOpOperationCallbackFactory.INSTANCE);
         ClusterConfig.setLocationConstraint(spec, deleteOp);
 
         /** construct empty sink operator */
@@ -234,8 +226,7 @@
         EmptySinkOperatorDescriptor emptySink4 = new EmptySinkOperatorDescriptor(spec);
         ClusterConfig.setLocationConstraint(spec, emptySink4);
 
-        ITuplePartitionComputerFactory partionFactory = new VertexIdPartitionComputerFactory(
-                new WritableSerializerDeserializerFactory(vertexIdClass));
+        ITuplePartitionComputerFactory partionFactory = getVertexPartitionComputerFactory();
         /** connect all operators **/
         spec.connect(new OneToOneConnectorDescriptor(spec), emptyTupleSource, 0, preSuperStep, 0);
         spec.connect(new OneToOneConnectorDescriptor(spec), preSuperStep, 0, scanner, 0);
@@ -321,10 +312,6 @@
         ITypeTraits[] typeTraits = new ITypeTraits[2];
         typeTraits[0] = new TypeTraits(false);
         typeTraits[1] = new TypeTraits(false);
-        ITreeIndexFrameFactory interiorFrameFactory = new BTreeNSMInteriorFrameFactory(new TypeAwareTupleWriterFactory(
-                typeTraits));
-        ITreeIndexFrameFactory leafFrameFactory = new BTreeNSMLeafFrameFactory(new TypeAwareTupleWriterFactory(
-                typeTraits));
         INullWriterFactory[] nullWriterFactories = new INullWriterFactory[2];
         nullWriterFactories[0] = VertexIdNullWriterFactory.INSTANCE;
         nullWriterFactories[1] = MsgListNullWriterFactory.INSTANCE;
@@ -338,11 +325,10 @@
                 vertexIdClass.getName(), MsgList.class.getName(), vertexIdClass.getName(), vertexClass.getName());
 
         IndexNestedLoopJoinFunctionUpdateOperatorDescriptor join = new IndexNestedLoopJoinFunctionUpdateOperatorDescriptor(
-                spec, storageManagerInterface, lcManagerProvider, fileSplitProvider, interiorFrameFactory,
-                leafFrameFactory, typeTraits, comparatorFactories, JobGenUtil.getForwardScan(iteration), keyFields,
-                keyFields, true, true, new BTreeDataflowHelperFactory(), true, nullWriterFactories, inputRdFactory, 5,
-                new ComputeUpdateFunctionFactory(confFactory), preHookFactory, null, rdUnnestedMessage, rdDummy,
-                rdPartialAggregate, rdInsert, rdDelete);
+                spec, storageManagerInterface, lcManagerProvider, fileSplitProvider, typeTraits, comparatorFactories,
+                JobGenUtil.getForwardScan(iteration), keyFields, keyFields, true, true, getIndexDataflowHelperFactory(), true,
+                nullWriterFactories, inputRdFactory, 5, new ComputeUpdateFunctionFactory(confFactory), preHookFactory,
+                null, rdUnnestedMessage, rdDummy, rdPartialAggregate, rdInsert, rdDelete);
         ClusterConfig.setLocationConstraint(spec, join);
 
         /**
@@ -418,8 +404,8 @@
         int[] fieldPermutation = new int[] { 0, 1 };
         TreeIndexInsertUpdateDeleteOperatorDescriptor insertOp = new TreeIndexInsertUpdateDeleteOperatorDescriptor(
                 spec, rdInsert, storageManagerInterface, lcManagerProvider, fileSplitProvider, typeTraits,
-                comparatorFactories, null, fieldPermutation, IndexOperation.INSERT, new BTreeDataflowHelperFactory(),
-                null, NoOpOperationCallbackFactory.INSTANCE);
+                comparatorFactories, null, fieldPermutation, IndexOperation.INSERT, getIndexDataflowHelperFactory(), null,
+                NoOpOperationCallbackFactory.INSTANCE);
         ClusterConfig.setLocationConstraint(spec, insertOp);
 
         /**
@@ -428,8 +414,8 @@
         int[] fieldPermutationDelete = new int[] { 0 };
         TreeIndexInsertUpdateDeleteOperatorDescriptor deleteOp = new TreeIndexInsertUpdateDeleteOperatorDescriptor(
                 spec, rdDelete, storageManagerInterface, lcManagerProvider, fileSplitProvider, typeTraits,
-                comparatorFactories, null, fieldPermutationDelete, IndexOperation.DELETE,
-                new BTreeDataflowHelperFactory(), null, NoOpOperationCallbackFactory.INSTANCE);
+                comparatorFactories, null, fieldPermutationDelete, IndexOperation.DELETE, getIndexDataflowHelperFactory(),
+                null, NoOpOperationCallbackFactory.INSTANCE);
         ClusterConfig.setLocationConstraint(spec, deleteOp);
 
         /** construct empty sink operator */
@@ -441,8 +427,7 @@
         ClusterConfig.setLocationConstraint(spec, emptySink4);
 
         ITuplePartitionComputerFactory unifyingPartitionComputerFactory = new MergePartitionComputerFactory();
-        ITuplePartitionComputerFactory partionFactory = new VertexIdPartitionComputerFactory(
-                new WritableSerializerDeserializerFactory(vertexIdClass));
+        ITuplePartitionComputerFactory partionFactory = getVertexPartitionComputerFactory();
 
         /** connect all operators **/
         spec.connect(new OneToOneConnectorDescriptor(spec), emptyTupleSource, 0, preSuperStep, 0);
diff --git a/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/util/PregelixHyracksIntegrationUtil.java b/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/util/PregelixHyracksIntegrationUtil.java
index 56fd676..73b053f 100644
--- a/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/util/PregelixHyracksIntegrationUtil.java
+++ b/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/util/PregelixHyracksIntegrationUtil.java
@@ -67,6 +67,7 @@
         ncConfig1.dataIPAddress = "127.0.0.1";
         ncConfig1.datasetIPAddress = "127.0.0.1";
         ncConfig1.nodeId = NC1_ID;
+        ncConfig1.ioDevices="dev1,dev2";
         ncConfig1.appNCMainClass = NCApplicationEntryPoint.class.getName();
         nc1 = new NodeControllerService(ncConfig1);
         nc1.start();
@@ -79,6 +80,7 @@
         ncConfig2.datasetIPAddress = "127.0.0.1";
         ncConfig2.nodeId = NC2_ID;
         ncConfig2.appNCMainClass = NCApplicationEntryPoint.class.getName();
+        ncConfig2.ioDevices="dev1,dev2";
         nc2 = new NodeControllerService(ncConfig2);
         nc2.start();
 
diff --git a/pregelix/pregelix-core/src/test/java/edu/uci/ics/pregelix/core/join/JoinTest.java b/pregelix/pregelix-core/src/test/java/edu/uci/ics/pregelix/core/join/JoinTest.java
index 24a96a3..4c7f91d 100644
--- a/pregelix/pregelix-core/src/test/java/edu/uci/ics/pregelix/core/join/JoinTest.java
+++ b/pregelix/pregelix-core/src/test/java/edu/uci/ics/pregelix/core/join/JoinTest.java
@@ -278,7 +278,7 @@
             typeTraits[i] = new TypeTraits(false);
         TreeIndexBulkLoadOperatorDescriptor writer = new TreeIndexBulkLoadOperatorDescriptor(spec,
                 storageManagerInterface, lcManagerProvider, fileSplitProvider, typeTraits, comparatorFactories, null,
-                fieldPermutation, DEFAULT_BTREE_FILL_FACTOR, false, 0, new BTreeDataflowHelperFactory(),
+                fieldPermutation, DEFAULT_BTREE_FILL_FACTOR, false, 0, false, new BTreeDataflowHelperFactory(),
                 NoOpOperationCallbackFactory.INSTANCE);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, writer, NC1_ID, NC2_ID);
 
diff --git a/pregelix/pregelix-dataflow-std/pom.xml b/pregelix/pregelix-dataflow-std/pom.xml
index 0a478d4..3604e57 100644
--- a/pregelix/pregelix-dataflow-std/pom.xml
+++ b/pregelix/pregelix-dataflow-std/pom.xml
@@ -143,6 +143,13 @@
 		</dependency>
 		<dependency>
 			<groupId>edu.uci.ics.hyracks</groupId>
+			<artifactId>hyracks-storage-am-lsm-btree</artifactId>
+			<version>0.2.7-SNAPSHOT</version>
+			<type>jar</type>
+			<scope>compile</scope>
+		</dependency>
+		<dependency>
+			<groupId>edu.uci.ics.hyracks</groupId>
 			<artifactId>hyracks-control-cc</artifactId>
 			<version>0.2.7-SNAPSHOT</version>
 			<type>jar</type>
diff --git a/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/IndexNestedLoopJoinFunctionUpdateOperatorDescriptor.java b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/IndexNestedLoopJoinFunctionUpdateOperatorDescriptor.java
index 0b3a7fe..3403afc 100644
--- a/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/IndexNestedLoopJoinFunctionUpdateOperatorDescriptor.java
+++ b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/IndexNestedLoopJoinFunctionUpdateOperatorDescriptor.java
@@ -23,10 +23,10 @@
 import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
 import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.api.job.JobSpecification;
 import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
 import edu.uci.ics.hyracks.storage.am.common.api.IIndexLifecycleManagerProvider;
-import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
 import edu.uci.ics.hyracks.storage.am.common.dataflow.AbstractTreeIndexOperatorDescriptor;
 import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
 import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallbackFactory;
@@ -91,8 +91,7 @@
 
     public IndexNestedLoopJoinFunctionUpdateOperatorDescriptor(JobSpecification spec,
             IStorageManagerInterface storageManager, IIndexLifecycleManagerProvider lcManagerProvider,
-            IFileSplitProvider fileSplitProvider, ITreeIndexFrameFactory interiorFrameFactory,
-            ITreeIndexFrameFactory leafFrameFactory, ITypeTraits[] typeTraits,
+            IFileSplitProvider fileSplitProvider, ITypeTraits[] typeTraits,
             IBinaryComparatorFactory[] comparatorFactories, boolean isForward, int[] lowKeyFields, int[] highKeyFields,
             boolean lowKeyInclusive, boolean highKeyInclusive, IIndexDataflowHelperFactory opHelperFactory,
             boolean isRightOuter, INullWriterFactory[] nullWriterFactories, IRecordDescriptorFactory inputRdFactory,
@@ -125,8 +124,7 @@
 
     public IndexNestedLoopJoinFunctionUpdateOperatorDescriptor(JobSpecification spec,
             IStorageManagerInterface storageManager, IIndexLifecycleManagerProvider lcManagerProvider,
-            IFileSplitProvider fileSplitProvider, ITreeIndexFrameFactory interiorFrameFactory,
-            ITreeIndexFrameFactory leafFrameFactory, ITypeTraits[] typeTraits,
+            IFileSplitProvider fileSplitProvider, ITypeTraits[] typeTraits,
             IBinaryComparatorFactory[] comparatorFactories, boolean isForward, int[] lowKeyFields, int[] highKeyFields,
             boolean lowKeyInclusive, boolean highKeyInclusive, IIndexDataflowHelperFactory opHelperFactory,
             boolean isSetUnion, IRecordDescriptorFactory inputRdFactory, int outputArity,
@@ -158,7 +156,7 @@
 
     @Override
     public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx,
-            IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) {
+            IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) throws HyracksDataException {
         if (isRightOuter) {
             INullWriter[] nullWriters = new INullWriter[nullWriterFactories.length];
             for (int i = 0; i < nullWriters.length; i++)
diff --git a/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/IndexNestedLoopJoinFunctionUpdateOperatorNodePushable.java b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/IndexNestedLoopJoinFunctionUpdateOperatorNodePushable.java
index bd45509..5156dbf 100644
--- a/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/IndexNestedLoopJoinFunctionUpdateOperatorNodePushable.java
+++ b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/IndexNestedLoopJoinFunctionUpdateOperatorNodePushable.java
@@ -28,15 +28,12 @@
 import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAppender;
 import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
 import edu.uci.ics.hyracks.dataflow.std.base.AbstractUnaryInputOperatorNodePushable;
-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.ITreeIndexAccessor;
-import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexCursor;
-import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrame;
+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.ITreeIndex;
 import edu.uci.ics.hyracks.storage.am.common.dataflow.AbstractTreeIndexOperatorDescriptor;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.TreeIndexDataflowHelper;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.IndexDataflowHelper;
 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.common.tuples.PermutingFrameTupleReference;
@@ -49,12 +46,12 @@
 import edu.uci.ics.pregelix.dataflow.util.UpdateBuffer;
 
 public class IndexNestedLoopJoinFunctionUpdateOperatorNodePushable extends AbstractUnaryInputOperatorNodePushable {
-    private TreeIndexDataflowHelper treeIndexOpHelper;
+    private IndexDataflowHelper treeIndexOpHelper;
     private FrameTupleAccessor accessor;
 
     private ByteBuffer writeBuffer;
     private FrameTupleAppender appender;
-    private BTree btree;
+    private ITreeIndex index;
     private PermutingFrameTupleReference lowKey;
     private PermutingFrameTupleReference highKey;
     private boolean lowKeyInclusive;
@@ -62,9 +59,8 @@
     private RangePredicate rangePred;
     private MultiComparator lowKeySearchCmp;
     private MultiComparator highKeySearchCmp;
-    private ITreeIndexCursor cursor;
-    private ITreeIndexFrame cursorFrame;
-    protected ITreeIndexAccessor indexAccessor;
+    private IIndexCursor cursor;
+    protected IIndexAccessor indexAccessor;
 
     private RecordDescriptor recDesc;
     private final IFrameWriter[] writers;
@@ -77,8 +73,9 @@
             IHyracksTaskContext ctx, int partition, IRecordDescriptorProvider recordDescProvider, boolean isForward,
             int[] lowKeyFields, int[] highKeyFields, boolean lowKeyInclusive, boolean highKeyInclusive,
             IUpdateFunctionFactory functionFactory, IRuntimeHookFactory preHookFactory,
-            IRuntimeHookFactory postHookFactory, IRecordDescriptorFactory inputRdFactory, int outputArity) {
-        treeIndexOpHelper = (TreeIndexDataflowHelper) opDesc.getIndexDataflowHelperFactory().createIndexDataflowHelper(
+            IRuntimeHookFactory postHookFactory, IRecordDescriptorFactory inputRdFactory, int outputArity)
+            throws HyracksDataException {
+        treeIndexOpHelper = (IndexDataflowHelper) opDesc.getIndexDataflowHelperFactory().createIndexDataflowHelper(
                 opDesc, ctx, partition);
         this.lowKeyInclusive = lowKeyInclusive;
         this.highKeyInclusive = highKeyInclusive;
@@ -99,7 +96,7 @@
     }
 
     protected void setCursor() {
-        cursor = new BTreeRangeSearchCursor((IBTreeLeafFrame) cursorFrame, true);
+        cursor = indexAccessor.createSearchCursor();
     }
 
     @Override
@@ -112,13 +109,11 @@
 
         try {
             treeIndexOpHelper.open();
-            btree = (BTree) treeIndexOpHelper.getIndexInstance();
-            cursorFrame = btree.getLeafFrameFactory().createFrame();
-            setCursor();
+            index = (ITreeIndex) treeIndexOpHelper.getIndexInstance();
 
             // TODO: Can we construct the multicmps using helper methods?
-            int lowKeySearchFields = btree.getComparatorFactories().length;
-            int highKeySearchFields = btree.getComparatorFactories().length;
+            int lowKeySearchFields = index.getComparatorFactories().length;
+            int highKeySearchFields = index.getComparatorFactories().length;
             if (lowKey != null)
                 lowKeySearchFields = lowKey.getFieldCount();
             if (highKey != null)
@@ -126,7 +121,7 @@
 
             IBinaryComparator[] lowKeySearchComparators = new IBinaryComparator[lowKeySearchFields];
             for (int i = 0; i < lowKeySearchFields; i++) {
-                lowKeySearchComparators[i] = btree.getComparatorFactories()[i].createBinaryComparator();
+                lowKeySearchComparators[i] = index.getComparatorFactories()[i].createBinaryComparator();
             }
             lowKeySearchCmp = new MultiComparator(lowKeySearchComparators);
 
@@ -135,10 +130,9 @@
             } else {
                 IBinaryComparator[] highKeySearchComparators = new IBinaryComparator[highKeySearchFields];
                 for (int i = 0; i < highKeySearchFields; i++) {
-                    highKeySearchComparators[i] = btree.getComparatorFactories()[i].createBinaryComparator();
+                    highKeySearchComparators[i] = index.getComparatorFactories()[i].createBinaryComparator();
                 }
                 highKeySearchCmp = new MultiComparator(highKeySearchComparators);
-
             }
 
             rangePred = new RangePredicate(null, null, lowKeyInclusive, highKeyInclusive, lowKeySearchCmp,
@@ -147,9 +141,10 @@
             appender = new FrameTupleAppender(treeIndexOpHelper.getTaskContext().getFrameSize());
             appender.reset(writeBuffer, true);
 
-            indexAccessor = btree.createAccessor(NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE);
-            cloneUpdateTb = new ArrayTupleBuilder(btree.getFieldCount());
-            updateBuffer.setFieldCount(btree.getFieldCount());
+            indexAccessor = index.createAccessor(NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE);
+            setCursor();
+            cloneUpdateTb = new ArrayTupleBuilder(index.getFieldCount());
+            updateBuffer.setFieldCount(index.getFieldCount());
         } catch (Exception e) {
             treeIndexOpHelper.close();
             throw new HyracksDataException(e);
@@ -201,7 +196,7 @@
             try {
                 cursor.close();
                 //batch update
-                updateBuffer.updateBTree(indexAccessor);
+                updateBuffer.updateIndex(indexAccessor);
             } catch (Exception e) {
                 throw new HyracksDataException(e);
             }
@@ -226,4 +221,4 @@
         writers[index] = writer;
     }
 
-}
\ No newline at end of file
+}
diff --git a/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/IndexNestedLoopJoinOperatorDescriptor.java b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/IndexNestedLoopJoinOperatorDescriptor.java
index 6dc713c..440ae86 100644
--- a/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/IndexNestedLoopJoinOperatorDescriptor.java
+++ b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/IndexNestedLoopJoinOperatorDescriptor.java
@@ -87,8 +87,7 @@
 
     public IndexNestedLoopJoinOperatorDescriptor(JobSpecification spec, RecordDescriptor recDesc,
             IStorageManagerInterface storageManager, IIndexLifecycleManagerProvider lcManagerProvider,
-            IFileSplitProvider fileSplitProvider, ITreeIndexFrameFactory interiorFrameFactory,
-            ITreeIndexFrameFactory leafFrameFactory, ITypeTraits[] typeTraits,
+            IFileSplitProvider fileSplitProvider, ITypeTraits[] typeTraits,
             IBinaryComparatorFactory[] comparatorFactories, boolean isForward, int[] lowKeyFields, int[] highKeyFields,
             boolean lowKeyInclusive, boolean highKeyInclusive, IIndexDataflowHelperFactory opHelperFactory,
             boolean isSetUnion) {
diff --git a/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/IndexNestedLoopJoinOperatorNodePushable.java b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/IndexNestedLoopJoinOperatorNodePushable.java
index d3114d2..2ddca90 100644
--- a/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/IndexNestedLoopJoinOperatorNodePushable.java
+++ b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/IndexNestedLoopJoinOperatorNodePushable.java
@@ -29,18 +29,18 @@
 import edu.uci.ics.hyracks.dataflow.common.comm.util.FrameUtils;
 import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
 import edu.uci.ics.hyracks.dataflow.std.base.AbstractUnaryInputUnaryOutputOperatorNodePushable;
-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.common.api.IIndexAccessor;
 import edu.uci.ics.hyracks.storage.am.common.api.IIndexCursor;
-import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexAccessor;
+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.TreeIndexDataflowHelper;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.IndexDataflowHelper;
 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.common.tuples.PermutingFrameTupleReference;
 
 public class IndexNestedLoopJoinOperatorNodePushable extends AbstractUnaryInputUnaryOutputOperatorNodePushable {
-    private TreeIndexDataflowHelper treeIndexOpHelper;
+    private IndexDataflowHelper treeIndexOpHelper;
     private FrameTupleAccessor accessor;
 
     private ByteBuffer writeBuffer;
@@ -48,7 +48,7 @@
     private ArrayTupleBuilder tb;
     private DataOutput dos;
 
-    private BTree btree;
+    private ITreeIndex index;
     private PermutingFrameTupleReference lowKey;
     private PermutingFrameTupleReference highKey;
     private boolean lowKeyInclusive;
@@ -57,7 +57,7 @@
     private MultiComparator lowKeySearchCmp;
     private MultiComparator highKeySearchCmp;
     private IIndexCursor cursor;
-    protected ITreeIndexAccessor indexAccessor;
+    protected IIndexAccessor indexAccessor;
 
     private RecordDescriptor recDesc;
     private final RecordDescriptor inputRecDesc;
@@ -66,7 +66,7 @@
             int partition, IRecordDescriptorProvider recordDescProvider, boolean isForward, int[] lowKeyFields,
             int[] highKeyFields, boolean lowKeyInclusive, boolean highKeyInclusive) {
         inputRecDesc = recordDescProvider.getInputRecordDescriptor(opDesc.getActivityId(), 0);
-        treeIndexOpHelper = (TreeIndexDataflowHelper) opDesc.getIndexDataflowHelperFactory().createIndexDataflowHelper(
+        treeIndexOpHelper = (IndexDataflowHelper) opDesc.getIndexDataflowHelperFactory().createIndexDataflowHelper(
                 opDesc, ctx, partition);
         this.lowKeyInclusive = lowKeyInclusive;
         this.highKeyInclusive = highKeyInclusive;
@@ -91,11 +91,11 @@
 
         try {
             treeIndexOpHelper.open();
-            btree = (BTree) treeIndexOpHelper.getIndexInstance();
+            index = (ITreeIndex) treeIndexOpHelper.getIndexInstance();
             writer.open();
 
-            int lowKeySearchFields = btree.getComparatorFactories().length;
-            int highKeySearchFields = btree.getComparatorFactories().length;
+            int lowKeySearchFields = index.getComparatorFactories().length;
+            int highKeySearchFields = index.getComparatorFactories().length;
             if (lowKey != null)
                 lowKeySearchFields = lowKey.getFieldCount();
             if (highKey != null)
@@ -103,7 +103,7 @@
 
             IBinaryComparator[] lowKeySearchComparators = new IBinaryComparator[lowKeySearchFields];
             for (int i = 0; i < lowKeySearchFields; i++) {
-                lowKeySearchComparators[i] = btree.getComparatorFactories()[i].createBinaryComparator();
+                lowKeySearchComparators[i] = index.getComparatorFactories()[i].createBinaryComparator();
             }
             lowKeySearchCmp = new MultiComparator(lowKeySearchComparators);
 
@@ -112,7 +112,7 @@
             } else {
                 IBinaryComparator[] highKeySearchComparators = new IBinaryComparator[highKeySearchFields];
                 for (int i = 0; i < highKeySearchFields; i++) {
-                    highKeySearchComparators[i] = btree.getComparatorFactories()[i].createBinaryComparator();
+                    highKeySearchComparators[i] = index.getComparatorFactories()[i].createBinaryComparator();
                 }
                 highKeySearchCmp = new MultiComparator(highKeySearchComparators);
             }
@@ -120,11 +120,11 @@
             rangePred = new RangePredicate(null, null, lowKeyInclusive, highKeyInclusive, lowKeySearchCmp,
                     highKeySearchCmp);
             writeBuffer = treeIndexOpHelper.getTaskContext().allocateFrame();
-            tb = new ArrayTupleBuilder(inputRecDesc.getFields().length + btree.getFieldCount());
+            tb = new ArrayTupleBuilder(inputRecDesc.getFields().length + index.getFieldCount());
             dos = tb.getDataOutput();
             appender = new FrameTupleAppender(treeIndexOpHelper.getTaskContext().getFrameSize());
             appender.reset(writeBuffer, true);
-            indexAccessor = btree.createAccessor(NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE);
+            indexAccessor = index.createAccessor(NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE);
             setCursor();
         } catch (Exception e) {
             treeIndexOpHelper.close();
diff --git a/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/IndexNestedLoopRightOuterJoinFunctionUpdateOperatorNodePushable.java b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/IndexNestedLoopRightOuterJoinFunctionUpdateOperatorNodePushable.java
index 289128e..4ca7533 100644
--- a/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/IndexNestedLoopRightOuterJoinFunctionUpdateOperatorNodePushable.java
+++ b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/IndexNestedLoopRightOuterJoinFunctionUpdateOperatorNodePushable.java
@@ -30,15 +30,12 @@
 import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAppender;
 import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
 import edu.uci.ics.hyracks.dataflow.std.base.AbstractUnaryInputOperatorNodePushable;
-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.ITreeIndexAccessor;
-import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexCursor;
-import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrame;
+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.ITreeIndex;
 import edu.uci.ics.hyracks.storage.am.common.dataflow.AbstractTreeIndexOperatorDescriptor;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.TreeIndexDataflowHelper;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.IndexDataflowHelper;
 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.common.tuples.PermutingFrameTupleReference;
@@ -52,7 +49,7 @@
 
 public class IndexNestedLoopRightOuterJoinFunctionUpdateOperatorNodePushable extends
         AbstractUnaryInputOperatorNodePushable {
-    private TreeIndexDataflowHelper treeIndexOpHelper;
+    private IndexDataflowHelper treeIndexOpHelper;
     private FrameTupleAccessor accessor;
 
     private ByteBuffer writeBuffer;
@@ -60,13 +57,12 @@
     private ArrayTupleBuilder nullTupleBuilder;
     private DataOutput dos;
 
-    private BTree btree;
+    private ITreeIndex index;
     private RangePredicate rangePred;
     private MultiComparator lowKeySearchCmp;
     private MultiComparator highKeySearchCmp;
-    private ITreeIndexCursor cursor;
-    private ITreeIndexFrame cursorFrame;
-    protected ITreeIndexAccessor indexAccessor;
+    private IIndexCursor cursor;
+    protected IIndexAccessor indexAccessor;
 
     private RecordDescriptor recDesc;
     private final RecordDescriptor inputRecDesc;
@@ -88,9 +84,9 @@
             IHyracksTaskContext ctx, int partition, IRecordDescriptorProvider recordDescProvider, boolean isForward,
             int[] lowKeyFields, int[] highKeyFields, INullWriter[] nullWriter, IUpdateFunctionFactory functionFactory,
             IRuntimeHookFactory preHookFactory, IRuntimeHookFactory postHookFactory,
-            IRecordDescriptorFactory inputRdFactory, int outputArity) {
+            IRecordDescriptorFactory inputRdFactory, int outputArity) throws HyracksDataException {
         inputRecDesc = recordDescProvider.getInputRecordDescriptor(opDesc.getActivityId(), 0);
-        treeIndexOpHelper = (TreeIndexDataflowHelper) opDesc.getIndexDataflowHelperFactory().createIndexDataflowHelper(
+        treeIndexOpHelper = (IndexDataflowHelper) opDesc.getIndexDataflowHelperFactory().createIndexDataflowHelper(
                 opDesc, ctx, partition);
         this.recDesc = recordDescProvider.getInputRecordDescriptor(opDesc.getActivityId(), 0);
 
@@ -111,7 +107,7 @@
     }
 
     protected void setCursor() {
-        cursor = new BTreeRangeSearchCursor((IBTreeLeafFrame) cursorFrame, true);
+        cursor = indexAccessor.createSearchCursor();
     }
 
     @Override
@@ -124,18 +120,16 @@
 
         try {
             treeIndexOpHelper.open();
-            btree = (BTree) treeIndexOpHelper.getIndexInstance();
-            cursorFrame = btree.getLeafFrameFactory().createFrame();
-            setCursor();
+            index = (ITreeIndex) treeIndexOpHelper.getIndexInstance();
 
             // construct range predicate
             // TODO: Can we construct the multicmps using helper methods?
-            int lowKeySearchFields = btree.getComparatorFactories().length;
-            int highKeySearchFields = btree.getComparatorFactories().length;
+            int lowKeySearchFields = index.getComparatorFactories().length;
+            int highKeySearchFields = index.getComparatorFactories().length;
 
             IBinaryComparator[] lowKeySearchComparators = new IBinaryComparator[lowKeySearchFields];
             for (int i = 0; i < lowKeySearchFields; i++) {
-                lowKeySearchComparators[i] = btree.getComparatorFactories()[i].createBinaryComparator();
+                lowKeySearchComparators[i] = index.getComparatorFactories()[i].createBinaryComparator();
             }
             lowKeySearchCmp = new MultiComparator(lowKeySearchComparators);
 
@@ -144,7 +138,7 @@
             } else {
                 IBinaryComparator[] highKeySearchComparators = new IBinaryComparator[highKeySearchFields];
                 for (int i = 0; i < highKeySearchFields; i++) {
-                    highKeySearchComparators[i] = btree.getComparatorFactories()[i].createBinaryComparator();
+                    highKeySearchComparators[i] = index.getComparatorFactories()[i].createBinaryComparator();
                 }
                 highKeySearchCmp = new MultiComparator(highKeySearchComparators);
             }
@@ -164,7 +158,8 @@
             appender = new FrameTupleAppender(treeIndexOpHelper.getTaskContext().getFrameSize());
             appender.reset(writeBuffer, true);
 
-            indexAccessor = btree.createAccessor(NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE);
+            indexAccessor = index.createAccessor(NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE);
+            setCursor();
 
             /** set the search cursor */
             rangePred.setLowKey(null, true);
@@ -179,8 +174,8 @@
                 match = false;
             }
 
-            cloneUpdateTb = new ArrayTupleBuilder(btree.getFieldCount());
-            updateBuffer.setFieldCount(btree.getFieldCount());
+            cloneUpdateTb = new ArrayTupleBuilder(index.getFieldCount());
+            updateBuffer.setFieldCount(index.getFieldCount());
         } catch (Exception e) {
             treeIndexOpHelper.close();
             throw new HyracksDataException(e);
@@ -239,7 +234,7 @@
             try {
                 cursor.close();
                 //batch update
-                updateBuffer.updateBTree(indexAccessor);
+                updateBuffer.updateIndex(indexAccessor);
             } catch (Exception e) {
                 throw new HyracksDataException(e);
             }
diff --git a/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/IndexNestedLoopRightOuterJoinOperatorNodePushable.java b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/IndexNestedLoopRightOuterJoinOperatorNodePushable.java
index 6f7643e..dd6ee3c 100644
--- a/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/IndexNestedLoopRightOuterJoinOperatorNodePushable.java
+++ b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/IndexNestedLoopRightOuterJoinOperatorNodePushable.java
@@ -30,22 +30,19 @@
 import edu.uci.ics.hyracks.dataflow.common.comm.util.FrameUtils;
 import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
 import edu.uci.ics.hyracks.dataflow.std.base.AbstractUnaryInputUnaryOutputOperatorNodePushable;
-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.ITreeIndexAccessor;
-import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexCursor;
-import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrame;
+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.ITreeIndex;
 import edu.uci.ics.hyracks.storage.am.common.dataflow.AbstractTreeIndexOperatorDescriptor;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.TreeIndexDataflowHelper;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.IndexDataflowHelper;
 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.common.tuples.PermutingFrameTupleReference;
 
 public class IndexNestedLoopRightOuterJoinOperatorNodePushable extends
         AbstractUnaryInputUnaryOutputOperatorNodePushable {
-    private TreeIndexDataflowHelper treeIndexOpHelper;
+    private IndexDataflowHelper treeIndexOpHelper;
     private FrameTupleAccessor accessor;
 
     private ByteBuffer writeBuffer;
@@ -53,14 +50,13 @@
     private ArrayTupleBuilder tb;
     private DataOutput dos;
 
-    private BTree btree;
+    private ITreeIndex index;
     private boolean isForward;
     private RangePredicate rangePred;
     private MultiComparator lowKeySearchCmp;
     private MultiComparator highKeySearchCmp;
-    private ITreeIndexCursor cursor;
-    private ITreeIndexFrame cursorFrame;
-    protected ITreeIndexAccessor indexAccessor;
+    private IIndexCursor cursor;
+    protected IIndexAccessor indexAccessor;
 
     private RecordDescriptor recDesc;
     private final RecordDescriptor inputRecDesc;
@@ -76,7 +72,7 @@
             IHyracksTaskContext ctx, int partition, IRecordDescriptorProvider recordDescProvider, boolean isForward,
             int[] lowKeyFields, int[] highKeyFields, INullWriter[] nullWriter) {
         inputRecDesc = recordDescProvider.getInputRecordDescriptor(opDesc.getActivityId(), 0);
-        treeIndexOpHelper = (TreeIndexDataflowHelper) opDesc.getIndexDataflowHelperFactory().createIndexDataflowHelper(
+        treeIndexOpHelper = (IndexDataflowHelper) opDesc.getIndexDataflowHelperFactory().createIndexDataflowHelper(
                 opDesc, ctx, partition);
         this.isForward = isForward;
         this.recDesc = recordDescProvider.getInputRecordDescriptor(opDesc.getActivityId(), 0);
@@ -93,7 +89,7 @@
     }
 
     protected void setCursor() {
-        cursor = new BTreeRangeSearchCursor((IBTreeLeafFrame) cursorFrame, false);
+        cursor = indexAccessor.createSearchCursor();
     }
 
     @Override
@@ -101,19 +97,17 @@
         accessor = new FrameTupleAccessor(treeIndexOpHelper.getTaskContext().getFrameSize(), recDesc);
         try {
             treeIndexOpHelper.open();
-            btree = (BTree) treeIndexOpHelper.getIndexInstance();
-            cursorFrame = btree.getLeafFrameFactory().createFrame();
-            setCursor();
+            index = (ITreeIndex) treeIndexOpHelper.getIndexInstance();
             writer.open();
 
             // construct range predicate
             // TODO: Can we construct the multicmps using helper methods?
-            int lowKeySearchFields = btree.getComparatorFactories().length;
-            int highKeySearchFields = btree.getComparatorFactories().length;
+            int lowKeySearchFields = index.getComparatorFactories().length;
+            int highKeySearchFields = index.getComparatorFactories().length;
 
             IBinaryComparator[] lowKeySearchComparators = new IBinaryComparator[lowKeySearchFields];
             for (int i = 0; i < lowKeySearchFields; i++) {
-                lowKeySearchComparators[i] = btree.getComparatorFactories()[i].createBinaryComparator();
+                lowKeySearchComparators[i] = index.getComparatorFactories()[i].createBinaryComparator();
             }
             lowKeySearchCmp = new MultiComparator(lowKeySearchComparators);
 
@@ -122,7 +116,7 @@
             } else {
                 IBinaryComparator[] highKeySearchComparators = new IBinaryComparator[highKeySearchFields];
                 for (int i = 0; i < highKeySearchFields; i++) {
-                    highKeySearchComparators[i] = btree.getComparatorFactories()[i].createBinaryComparator();
+                    highKeySearchComparators[i] = index.getComparatorFactories()[i].createBinaryComparator();
                 }
                 highKeySearchCmp = new MultiComparator(highKeySearchComparators);
 
@@ -131,12 +125,13 @@
             rangePred = new RangePredicate(null, null, true, true, lowKeySearchCmp, highKeySearchCmp);
 
             writeBuffer = treeIndexOpHelper.getTaskContext().allocateFrame();
-            tb = new ArrayTupleBuilder(inputRecDesc.getFields().length + btree.getFieldCount());
+            tb = new ArrayTupleBuilder(inputRecDesc.getFields().length + index.getFieldCount());
             dos = tb.getDataOutput();
             appender = new FrameTupleAppender(treeIndexOpHelper.getTaskContext().getFrameSize());
             appender.reset(writeBuffer, true);
 
-            indexAccessor = btree.createAccessor(NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE);
+            indexAccessor = index.createAccessor(NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE);
+            setCursor();
 
             /** set the search cursor */
             rangePred.setLowKey(null, true);
diff --git a/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/IndexNestedLoopSetUnionFunctionUpdateOperatorNodePushable.java b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/IndexNestedLoopSetUnionFunctionUpdateOperatorNodePushable.java
index c978614..3cebfb8 100644
--- a/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/IndexNestedLoopSetUnionFunctionUpdateOperatorNodePushable.java
+++ b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/IndexNestedLoopSetUnionFunctionUpdateOperatorNodePushable.java
@@ -28,15 +28,12 @@
 import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAppender;
 import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
 import edu.uci.ics.hyracks.dataflow.std.base.AbstractUnaryInputOperatorNodePushable;
-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.ITreeIndexAccessor;
-import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexCursor;
-import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrame;
+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.ITreeIndex;
 import edu.uci.ics.hyracks.storage.am.common.dataflow.AbstractTreeIndexOperatorDescriptor;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.TreeIndexDataflowHelper;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.IndexDataflowHelper;
 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.common.tuples.PermutingFrameTupleReference;
@@ -49,19 +46,18 @@
 import edu.uci.ics.pregelix.dataflow.util.UpdateBuffer;
 
 public class IndexNestedLoopSetUnionFunctionUpdateOperatorNodePushable extends AbstractUnaryInputOperatorNodePushable {
-    private TreeIndexDataflowHelper treeIndexOpHelper;
+    private IndexDataflowHelper treeIndexOpHelper;
     private FrameTupleAccessor accessor;
 
     private ByteBuffer writeBuffer;
     private FrameTupleAppender appender;
 
-    private BTree btree;
+    private ITreeIndex index;
     private boolean isForward;
     private RangePredicate rangePred;
     private MultiComparator lowKeySearchCmp;
-    private ITreeIndexCursor cursor;
-    private ITreeIndexFrame cursorFrame;
-    protected ITreeIndexAccessor indexAccessor;
+    private IIndexCursor cursor;
+    protected IIndexAccessor indexAccessor;
 
     private RecordDescriptor recDesc;
     private PermutingFrameTupleReference lowKey;
@@ -80,8 +76,8 @@
             IHyracksTaskContext ctx, int partition, IRecordDescriptorProvider recordDescProvider, boolean isForward,
             int[] lowKeyFields, int[] highKeyFields, IUpdateFunctionFactory functionFactory,
             IRuntimeHookFactory preHookFactory, IRuntimeHookFactory postHookFactory,
-            IRecordDescriptorFactory inputRdFactory, int outputArity) {
-        treeIndexOpHelper = (TreeIndexDataflowHelper) opDesc.getIndexDataflowHelperFactory().createIndexDataflowHelper(
+            IRecordDescriptorFactory inputRdFactory, int outputArity) throws HyracksDataException {
+        treeIndexOpHelper = (IndexDataflowHelper) opDesc.getIndexDataflowHelperFactory().createIndexDataflowHelper(
                 opDesc, ctx, partition);
         this.isForward = isForward;
         this.recDesc = recordDescProvider.getInputRecordDescriptor(opDesc.getActivityId(), 0);
@@ -102,7 +98,7 @@
     }
 
     protected void setCursor() {
-        cursor = new BTreeRangeSearchCursor((IBTreeLeafFrame) cursorFrame, true);
+        cursor = indexAccessor.createSearchCursor();
     }
 
     @Override
@@ -112,15 +108,13 @@
 
         try {
             treeIndexOpHelper.open();
-            btree = (BTree) treeIndexOpHelper.getIndexInstance();
-            cursorFrame = btree.getLeafFrameFactory().createFrame();
-            setCursor();
+            index = (ITreeIndex) treeIndexOpHelper.getIndexInstance();
 
             rangePred = new RangePredicate(null, null, true, true, null, null);
-            int lowKeySearchFields = btree.getComparatorFactories().length;
+            int lowKeySearchFields = index.getComparatorFactories().length;
             IBinaryComparator[] lowKeySearchComparators = new IBinaryComparator[lowKeySearchFields];
             for (int i = 0; i < lowKeySearchFields; i++) {
-                lowKeySearchComparators[i] = btree.getComparatorFactories()[i].createBinaryComparator();
+                lowKeySearchComparators[i] = index.getComparatorFactories()[i].createBinaryComparator();
             }
             lowKeySearchCmp = new MultiComparator(lowKeySearchComparators);
 
@@ -128,7 +122,8 @@
             appender = new FrameTupleAppender(treeIndexOpHelper.getTaskContext().getFrameSize());
             appender.reset(writeBuffer, true);
 
-            indexAccessor = btree.createAccessor(NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE);
+            indexAccessor = index.createAccessor(NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE);
+            setCursor();
 
             /** set the search cursor */
             rangePred.setLowKey(null, true);
@@ -142,8 +137,8 @@
                 currentTopTuple = cursor.getTuple();
                 match = false;
             }
-            cloneUpdateTb = new ArrayTupleBuilder(btree.getFieldCount());
-            updateBuffer.setFieldCount(btree.getFieldCount());
+            cloneUpdateTb = new ArrayTupleBuilder(index.getFieldCount());
+            updateBuffer.setFieldCount(index.getFieldCount());
         } catch (Exception e) {
             treeIndexOpHelper.close();
             throw new HyracksDataException(e);
@@ -210,7 +205,7 @@
                 cursor.close();
 
                 //batch update
-                updateBuffer.updateBTree(indexAccessor);
+                updateBuffer.updateIndex(indexAccessor);
             } catch (Exception e) {
                 throw new HyracksDataException(e);
             }
@@ -256,4 +251,4 @@
     public void setOutputFrameWriter(int index, IFrameWriter writer, RecordDescriptor recordDesc) {
         writers[index] = writer;
     }
-}
\ No newline at end of file
+}
diff --git a/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/IndexNestedLoopSetUnionOperatorNodePushable.java b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/IndexNestedLoopSetUnionOperatorNodePushable.java
index eef7c85..bbe2764 100644
--- a/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/IndexNestedLoopSetUnionOperatorNodePushable.java
+++ b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/IndexNestedLoopSetUnionOperatorNodePushable.java
@@ -29,21 +29,18 @@
 import edu.uci.ics.hyracks.dataflow.common.comm.util.FrameUtils;
 import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
 import edu.uci.ics.hyracks.dataflow.std.base.AbstractUnaryInputUnaryOutputOperatorNodePushable;
-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.ITreeIndexAccessor;
-import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexCursor;
-import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrame;
+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.ITreeIndex;
 import edu.uci.ics.hyracks.storage.am.common.dataflow.AbstractTreeIndexOperatorDescriptor;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.TreeIndexDataflowHelper;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.IndexDataflowHelper;
 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.common.tuples.PermutingFrameTupleReference;
 
 public class IndexNestedLoopSetUnionOperatorNodePushable extends AbstractUnaryInputUnaryOutputOperatorNodePushable {
-    private TreeIndexDataflowHelper treeIndexOpHelper;
+    private IndexDataflowHelper treeIndexOpHelper;
     private FrameTupleAccessor accessor;
 
     private ByteBuffer writeBuffer;
@@ -51,12 +48,11 @@
     private ArrayTupleBuilder tb;
     private DataOutput dos;
 
-    private BTree btree;
+    private ITreeIndex index;
     private RangePredicate rangePred;
     private MultiComparator lowKeySearchCmp;
-    private ITreeIndexCursor cursor;
-    private ITreeIndexFrame cursorFrame;
-    protected ITreeIndexAccessor indexAccessor;
+    private IIndexCursor cursor;
+    protected IIndexAccessor indexAccessor;
 
     private RecordDescriptor recDesc;
     private final RecordDescriptor inputRecDesc;
@@ -71,7 +67,7 @@
             IHyracksTaskContext ctx, int partition, IRecordDescriptorProvider recordDescProvider, boolean isForward,
             int[] lowKeyFields, int[] highKeyFields) {
         inputRecDesc = recordDescProvider.getInputRecordDescriptor(opDesc.getActivityId(), 0);
-        treeIndexOpHelper = (TreeIndexDataflowHelper) opDesc.getIndexDataflowHelperFactory().createIndexDataflowHelper(
+        treeIndexOpHelper = (IndexDataflowHelper) opDesc.getIndexDataflowHelperFactory().createIndexDataflowHelper(
                 opDesc, ctx, partition);
         this.recDesc = recordDescProvider.getInputRecordDescriptor(opDesc.getActivityId(), 0);
 
@@ -86,7 +82,7 @@
     }
 
     protected void setCursor() {
-        cursor = new BTreeRangeSearchCursor((IBTreeLeafFrame) cursorFrame, false);
+        cursor = indexAccessor.createSearchCursor();
     }
 
     @Override
@@ -95,26 +91,25 @@
 
         try {
             treeIndexOpHelper.open();
-            btree = (BTree) treeIndexOpHelper.getIndexInstance();
-            cursorFrame = btree.getLeafFrameFactory().createFrame();
-            setCursor();
+            index = (ITreeIndex) treeIndexOpHelper.getIndexInstance();
             writer.open();
 
             rangePred = new RangePredicate(null, null, true, true, null, null);
-            int lowKeySearchFields = btree.getComparatorFactories().length;
+            int lowKeySearchFields = index.getComparatorFactories().length;
             IBinaryComparator[] lowKeySearchComparators = new IBinaryComparator[lowKeySearchFields];
             for (int i = 0; i < lowKeySearchFields; i++) {
-                lowKeySearchComparators[i] = btree.getComparatorFactories()[i].createBinaryComparator();
+                lowKeySearchComparators[i] = index.getComparatorFactories()[i].createBinaryComparator();
             }
             lowKeySearchCmp = new MultiComparator(lowKeySearchComparators);
 
             writeBuffer = treeIndexOpHelper.getTaskContext().allocateFrame();
-            tb = new ArrayTupleBuilder(btree.getFieldCount());
+            tb = new ArrayTupleBuilder(index.getFieldCount());
             dos = tb.getDataOutput();
             appender = new FrameTupleAppender(treeIndexOpHelper.getTaskContext().getFrameSize());
             appender.reset(writeBuffer, true);
 
-            indexAccessor = btree.createAccessor(NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE);
+            indexAccessor = index.createAccessor(NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE);
+            setCursor();
 
             /** set the search cursor */
             rangePred.setLowKey(null, true);
diff --git a/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/TreeIndexBulkReLoadOperatorDescriptor.java b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/TreeIndexBulkReLoadOperatorDescriptor.java
index 1dc1529..4dd78c3 100644
--- a/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/TreeIndexBulkReLoadOperatorDescriptor.java
+++ b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/TreeIndexBulkReLoadOperatorDescriptor.java
@@ -42,9 +42,9 @@
     public TreeIndexBulkReLoadOperatorDescriptor(JobSpecification spec, IStorageManagerInterface storageManager,
             IIndexLifecycleManagerProvider lcManagerProvider, IFileSplitProvider fileSplitProvider,
             ITypeTraits[] typeTraits, IBinaryComparatorFactory[] comparatorFactories, int[] fieldPermutation,
-            float fillFactor, IIndexDataflowHelperFactory opHelperFactory) {
+            int[] bloomFilterFields, float fillFactor, IIndexDataflowHelperFactory opHelperFactory) {
         super(spec, 1, 0, null, storageManager, lcManagerProvider, fileSplitProvider, typeTraits, comparatorFactories,
-                fieldPermutation, opHelperFactory, null, false, new TransientLocalResourceFactoryProvider(),
+                bloomFilterFields, opHelperFactory, null, false, new TransientLocalResourceFactoryProvider(),
                 NoOpOperationCallbackFactory.INSTANCE, NoOpOperationCallbackFactory.INSTANCE);
         this.fieldPermutation = fieldPermutation;
 
diff --git a/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/TreeIndexBulkReLoadOperatorNodePushable.java b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/TreeIndexBulkReLoadOperatorNodePushable.java
index 208d41c..c4890e1 100644
--- a/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/TreeIndexBulkReLoadOperatorNodePushable.java
+++ b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/TreeIndexBulkReLoadOperatorNodePushable.java
@@ -29,13 +29,13 @@
 import edu.uci.ics.hyracks.storage.am.common.api.IndexException;
 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.common.dataflow.TreeIndexDataflowHelper;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.IndexDataflowHelper;
 import edu.uci.ics.hyracks.storage.am.common.tuples.PermutingFrameTupleReference;
 import edu.uci.ics.hyracks.storage.common.IStorageManagerInterface;
 
 public class TreeIndexBulkReLoadOperatorNodePushable extends AbstractUnaryInputSinkOperatorNodePushable {
     private final float fillFactor;
-    private final TreeIndexDataflowHelper treeIndexOpHelper;
+    private final IndexDataflowHelper treeIndexOpHelper;
     private final IIndexOperatorDescriptor opDesc;
     private final IRecordDescriptorProvider recordDescProvider;
     private final PermutingFrameTupleReference tuple = new PermutingFrameTupleReference();
@@ -49,7 +49,7 @@
             IStorageManagerInterface storageManager, IIndexLifecycleManagerProvider lcManagerProvider,
             IFileSplitProvider fileSplitProvider) {
         this.fillFactor = fillFactor;
-        treeIndexOpHelper = (TreeIndexDataflowHelper) opDesc.getIndexDataflowHelperFactory().createIndexDataflowHelper(
+        treeIndexOpHelper = (IndexDataflowHelper) opDesc.getIndexDataflowHelperFactory().createIndexDataflowHelper(
                 opDesc, ctx, partition);
         this.opDesc = opDesc;
         this.recordDescProvider = recordDescProvider;
@@ -60,11 +60,12 @@
     public void open() throws HyracksDataException {
         RecordDescriptor recDesc = recordDescProvider.getInputRecordDescriptor(opDesc.getActivityId(), 0);
         accessor = new FrameTupleAccessor(treeIndexOpHelper.getTaskContext().getFrameSize(), recDesc);
+        treeIndexOpHelper.destroy();
         treeIndexOpHelper.create();
         treeIndexOpHelper.open();
         try {
             index = (ITreeIndex) treeIndexOpHelper.getIndexInstance();
-            bulkLoader = index.createBulkLoader(fillFactor, false, 0);
+            bulkLoader = index.createBulkLoader(fillFactor, false, 0, false);
         } catch (Exception e) {
             // cleanup in case of failure
             treeIndexOpHelper.close();
@@ -99,5 +100,6 @@
 
     @Override
     public void fail() throws HyracksDataException {
+
     }
 }
\ No newline at end of file
diff --git a/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/BTreeSearchFunctionUpdateOperatorDescriptor.java b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/TreeSearchFunctionUpdateOperatorDescriptor.java
similarity index 90%
rename from pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/BTreeSearchFunctionUpdateOperatorDescriptor.java
rename to pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/TreeSearchFunctionUpdateOperatorDescriptor.java
index 784288f..31496a9 100644
--- a/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/BTreeSearchFunctionUpdateOperatorDescriptor.java
+++ b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/TreeSearchFunctionUpdateOperatorDescriptor.java
@@ -21,6 +21,7 @@
 import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
 import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.api.job.JobSpecification;
 import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
 import edu.uci.ics.hyracks.storage.am.common.api.IIndexLifecycleManagerProvider;
@@ -33,7 +34,7 @@
 import edu.uci.ics.pregelix.dataflow.std.base.IRuntimeHookFactory;
 import edu.uci.ics.pregelix.dataflow.std.base.IUpdateFunctionFactory;
 
-public class BTreeSearchFunctionUpdateOperatorDescriptor extends AbstractTreeIndexOperatorDescriptor {
+public class TreeSearchFunctionUpdateOperatorDescriptor extends AbstractTreeIndexOperatorDescriptor {
 
     private static final long serialVersionUID = 1L;
 
@@ -52,7 +53,7 @@
 
     private final int outputArity;
 
-    public BTreeSearchFunctionUpdateOperatorDescriptor(JobSpecification spec, RecordDescriptor recDesc,
+    public TreeSearchFunctionUpdateOperatorDescriptor(JobSpecification spec, RecordDescriptor recDesc,
             IStorageManagerInterface storageManager, IIndexLifecycleManagerProvider lcManagerProvider,
             IFileSplitProvider fileSplitProvider, ITypeTraits[] typeTraits,
             IBinaryComparatorFactory[] comparatorFactories, boolean isForward, int[] lowKeyFields, int[] highKeyFields,
@@ -83,9 +84,9 @@
 
     @Override
     public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx,
-            IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) {
-        return new BTreeSearchFunctionUpdateOperatorNodePushable(this, ctx, partition, recordDescProvider, isForward,
+            IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) throws HyracksDataException {
+        return new TreeSearchFunctionUpdateOperatorNodePushable(this, ctx, partition, recordDescProvider, isForward,
                 lowKeyFields, highKeyFields, lowKeyInclusive, highKeyInclusive, functionFactory, preHookFactory,
                 postHookFactory, inputRdFactory, outputArity);
     }
-}
\ No newline at end of file
+}
diff --git a/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/BTreeSearchFunctionUpdateOperatorNodePushable.java b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/TreeSearchFunctionUpdateOperatorNodePushable.java
similarity index 73%
rename from pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/BTreeSearchFunctionUpdateOperatorNodePushable.java
rename to pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/TreeSearchFunctionUpdateOperatorNodePushable.java
index 0216b52..bd85e3e 100644
--- a/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/BTreeSearchFunctionUpdateOperatorNodePushable.java
+++ b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/TreeSearchFunctionUpdateOperatorNodePushable.java
@@ -19,6 +19,7 @@
 
 import edu.uci.ics.hyracks.api.comm.IFrameWriter;
 import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
 import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
@@ -27,16 +28,13 @@
 import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAppender;
 import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
 import edu.uci.ics.hyracks.dataflow.std.base.AbstractUnaryInputOperatorNodePushable;
-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.btree.util.BTreeUtils;
-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.IIndexAccessor;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexCursor;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
 import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrame;
 import edu.uci.ics.hyracks.storage.am.common.dataflow.AbstractTreeIndexOperatorDescriptor;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.TreeIndexDataflowHelper;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.IndexDataflowHelper;
 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.common.tuples.PermutingFrameTupleReference;
@@ -48,8 +46,8 @@
 import edu.uci.ics.pregelix.dataflow.util.SearchKeyTupleReference;
 import edu.uci.ics.pregelix.dataflow.util.UpdateBuffer;
 
-public class BTreeSearchFunctionUpdateOperatorNodePushable extends AbstractUnaryInputOperatorNodePushable {
-    protected TreeIndexDataflowHelper treeIndexHelper;
+public class TreeSearchFunctionUpdateOperatorNodePushable extends AbstractUnaryInputOperatorNodePushable {
+    protected IndexDataflowHelper treeIndexHelper;
     protected FrameTupleAccessor accessor;
 
     protected ByteBuffer writeBuffer;
@@ -57,7 +55,7 @@
     protected ArrayTupleBuilder tb;
     protected DataOutput dos;
 
-    protected BTree btree;
+    protected ITreeIndex index;
     protected boolean isForward;
     protected PermutingFrameTupleReference lowKey;
     protected PermutingFrameTupleReference highKey;
@@ -66,9 +64,11 @@
     protected RangePredicate rangePred;
     protected MultiComparator lowKeySearchCmp;
     protected MultiComparator highKeySearchCmp;
-    protected ITreeIndexCursor cursor;
+    protected IIndexCursor cursor;
     protected ITreeIndexFrame cursorFrame;
-    protected ITreeIndexAccessor indexAccessor;
+    protected IIndexAccessor indexAccessor;
+    protected int[] lowKeyFields;
+    protected int[] highKeyFields;
 
     protected RecordDescriptor recDesc;
 
@@ -78,16 +78,19 @@
     private final UpdateBuffer updateBuffer;
     private final SearchKeyTupleReference tempTupleReference = new SearchKeyTupleReference();
 
-    public BTreeSearchFunctionUpdateOperatorNodePushable(AbstractTreeIndexOperatorDescriptor opDesc,
+    public TreeSearchFunctionUpdateOperatorNodePushable(AbstractTreeIndexOperatorDescriptor opDesc,
             IHyracksTaskContext ctx, int partition, IRecordDescriptorProvider recordDescProvider, boolean isForward,
             int[] lowKeyFields, int[] highKeyFields, boolean lowKeyInclusive, boolean highKeyInclusive,
             IUpdateFunctionFactory functionFactory, IRuntimeHookFactory preHookFactory,
-            IRuntimeHookFactory postHookFactory, IRecordDescriptorFactory inputRdFactory, int outputArity) {
-        treeIndexHelper = (TreeIndexDataflowHelper) opDesc.getIndexDataflowHelperFactory().createIndexDataflowHelper(
+            IRuntimeHookFactory postHookFactory, IRecordDescriptorFactory inputRdFactory, int outputArity)
+            throws HyracksDataException {
+        treeIndexHelper = (IndexDataflowHelper) opDesc.getIndexDataflowHelperFactory().createIndexDataflowHelper(
                 opDesc, ctx, partition);
         this.isForward = isForward;
         this.lowKeyInclusive = lowKeyInclusive;
         this.highKeyInclusive = highKeyInclusive;
+        this.lowKeyFields = lowKeyFields;
+        this.highKeyFields = highKeyFields;
         this.recDesc = recordDescProvider.getInputRecordDescriptor(opDesc.getActivityId(), 0);
         if (lowKeyFields != null && lowKeyFields.length > 0) {
             lowKey = new PermutingFrameTupleReference();
@@ -114,25 +117,46 @@
 
         try {
             treeIndexHelper.open();
-            btree = (BTree) treeIndexHelper.getIndexInstance();
-            cursorFrame = btree.getLeafFrameFactory().createFrame();
-            setCursor();
+            index = (ITreeIndex) treeIndexHelper.getIndexInstance();
+            cursorFrame = index.getLeafFrameFactory().createFrame();
 
             // Construct range predicate.
-            lowKeySearchCmp = BTreeUtils.getSearchMultiComparator(btree.getComparatorFactories(), lowKey);
-            highKeySearchCmp = BTreeUtils.getSearchMultiComparator(btree.getComparatorFactories(), highKey);
+            int lowKeySearchFields = index.getComparatorFactories().length;
+            int highKeySearchFields = index.getComparatorFactories().length;
+            if (lowKey != null)
+                lowKeySearchFields = lowKey.getFieldCount();
+            if (highKey != null)
+                highKeySearchFields = highKey.getFieldCount();
+
+            IBinaryComparator[] lowKeySearchComparators = new IBinaryComparator[lowKeySearchFields];
+            for (int i = 0; i < lowKeySearchFields; i++) {
+                lowKeySearchComparators[i] = index.getComparatorFactories()[i].createBinaryComparator();
+            }
+            lowKeySearchCmp = new MultiComparator(lowKeySearchComparators);
+
+            if (lowKeySearchFields == highKeySearchFields) {
+                highKeySearchCmp = lowKeySearchCmp;
+            } else {
+                IBinaryComparator[] highKeySearchComparators = new IBinaryComparator[highKeySearchFields];
+                for (int i = 0; i < highKeySearchFields; i++) {
+                    highKeySearchComparators[i] = index.getComparatorFactories()[i].createBinaryComparator();
+                }
+                highKeySearchCmp = new MultiComparator(highKeySearchComparators);
+            }
+
             rangePred = new RangePredicate(null, null, lowKeyInclusive, highKeyInclusive, lowKeySearchCmp,
                     highKeySearchCmp);
 
             writeBuffer = treeIndexHelper.getTaskContext().allocateFrame();
-            tb = new ArrayTupleBuilder(btree.getFieldCount());
+            tb = new ArrayTupleBuilder(index.getFieldCount());
             dos = tb.getDataOutput();
             appender = new FrameTupleAppender(treeIndexHelper.getTaskContext().getFrameSize());
             appender.reset(writeBuffer, true);
-            indexAccessor = btree.createAccessor(NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE);
+            indexAccessor = index.createAccessor(NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE);
+            setCursor();
 
-            cloneUpdateTb = new ArrayTupleBuilder(btree.getFieldCount());
-            updateBuffer.setFieldCount(btree.getFieldCount());
+            cloneUpdateTb = new ArrayTupleBuilder(index.getFieldCount());
+            updateBuffer.setFieldCount(index.getFieldCount());
         } catch (Exception e) {
             treeIndexHelper.close();
             throw new HyracksDataException(e);
@@ -140,7 +164,7 @@
     }
 
     protected void setCursor() {
-        cursor = new BTreeRangeSearchCursor((IBTreeLeafFrame) cursorFrame, false);
+        cursor = indexAccessor.createSearchCursor();
     }
 
     protected void writeSearchResults() throws Exception {
@@ -184,7 +208,7 @@
             try {
                 cursor.close();
                 //batch update
-                updateBuffer.updateBTree(indexAccessor);
+                updateBuffer.updateIndex(indexAccessor);
             } catch (Exception e) {
                 throw new HyracksDataException(e);
             }
@@ -209,4 +233,4 @@
         writers[index] = writer;
     }
 
-}
\ No newline at end of file
+}
diff --git a/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/util/CopyUpdateUtil.java b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/util/CopyUpdateUtil.java
index b141eaf..8709301 100644
--- a/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/util/CopyUpdateUtil.java
+++ b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/util/CopyUpdateUtil.java
@@ -19,15 +19,15 @@
 import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
 import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
 import edu.uci.ics.hyracks.storage.am.btree.impls.RangePredicate;
-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.IIndexAccessor;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexCursor;
 import edu.uci.ics.hyracks.storage.am.common.api.IndexException;
 
 public class CopyUpdateUtil {
 
     public static void copyUpdate(SearchKeyTupleReference tempTupleReference, ITupleReference frameTuple,
-            UpdateBuffer updateBuffer, ArrayTupleBuilder cloneUpdateTb, ITreeIndexAccessor indexAccessor,
-            ITreeIndexCursor cursor, RangePredicate rangePred) throws HyracksDataException, IndexException {
+            UpdateBuffer updateBuffer, ArrayTupleBuilder cloneUpdateTb, IIndexAccessor indexAccessor,
+            IIndexCursor cursor, RangePredicate rangePred) throws HyracksDataException, IndexException {
         if (cloneUpdateTb.getSize() > 0) {
             int[] fieldEndOffsets = cloneUpdateTb.getFieldEndOffsets();
             int srcStart = fieldEndOffsets[0];
@@ -46,7 +46,7 @@
                 //release the cursor/latch
                 cursor.close();
                 //batch update
-                updateBuffer.updateBTree(indexAccessor);
+                updateBuffer.updateIndex(indexAccessor);
                 //try append the to-be-updated tuple again
                 if (!updateBuffer.appendTuple(cloneUpdateTb)) {
                     throw new HyracksDataException("cannot append tuple builder!");
diff --git a/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/util/UpdateBuffer.java b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/util/UpdateBuffer.java
index 327178c..b2be366 100644
--- a/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/util/UpdateBuffer.java
+++ b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/util/UpdateBuffer.java
@@ -25,7 +25,7 @@
 import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAppender;
 import edu.uci.ics.hyracks.dataflow.common.data.accessors.FrameTupleReference;
-import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexAccessor;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexAccessor;
 import edu.uci.ics.hyracks.storage.am.common.api.IndexException;
 
 /**
@@ -44,7 +44,7 @@
     private final int frameSize;
     private IFrameTupleAccessor fta;
 
-    public UpdateBuffer(int numPages, IHyracksTaskContext ctx, int fieldCount) {
+    public UpdateBuffer(int numPages, IHyracksTaskContext ctx, int fieldCount) throws HyracksDataException {
         this.appender = new FrameTupleAppender(ctx.getFrameSize());
         ByteBuffer buffer = ctx.allocateFrame();
         this.buffers.add(buffer);
@@ -55,7 +55,7 @@
         this.fta = new UpdateBufferTupleAccessor(frameSize, fieldCount);
     }
 
-    public UpdateBuffer(IHyracksTaskContext ctx, int fieldCount) {
+    public UpdateBuffer(IHyracksTaskContext ctx, int fieldCount) throws HyracksDataException {
         //by default, the update buffer has 1000 pages
         this(1000, ctx, fieldCount);
     }
@@ -87,7 +87,7 @@
         }
     }
 
-    public void updateBTree(ITreeIndexAccessor bta) throws HyracksDataException, IndexException {
+    public void updateIndex(IIndexAccessor bta) throws HyracksDataException, IndexException {
         // batch update
         for (int i = 0; i <= currentInUse; i++) {
             ByteBuffer buffer = buffers.get(i);
@@ -104,7 +104,7 @@
         appender.reset(buffer, true);
     }
 
-    private void allocate(int index) {
+    private void allocate(int index) throws HyracksDataException {
         if (index >= buffers.size()) {
             buffers.add(ctx.allocateFrame());
         }
diff --git a/pregelix/pregelix-dataflow/pom.xml b/pregelix/pregelix-dataflow/pom.xml
index 962c9f6..2828451 100644
--- a/pregelix/pregelix-dataflow/pom.xml
+++ b/pregelix/pregelix-dataflow/pom.xml
@@ -1,18 +1,14 @@
-<!--
- ! 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.
- !-->
-<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+<!-- ! 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. ! -->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+	xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
 	<modelVersion>4.0.0</modelVersion>
 	<artifactId>pregelix-dataflow</artifactId>
 	<packaging>jar</packaging>
@@ -134,6 +130,13 @@
 		</dependency>
 		<dependency>
 			<groupId>edu.uci.ics.hyracks</groupId>
+			<artifactId>hyracks-storage-am-lsm-common</artifactId>
+			<version>0.2.7-SNAPSHOT</version>
+			<type>jar</type>
+			<scope>compile</scope>
+		</dependency>
+		<dependency>
+			<groupId>edu.uci.ics.hyracks</groupId>
 			<artifactId>hyracks-control-cc</artifactId>
 			<version>0.2.7-SNAPSHOT</version>
 			<type>jar</type>
diff --git a/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/MaterializingReadOperatorDescriptor.java b/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/MaterializingReadOperatorDescriptor.java
index 86da3bf..ca8f190 100644
--- a/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/MaterializingReadOperatorDescriptor.java
+++ b/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/MaterializingReadOperatorDescriptor.java
@@ -38,7 +38,8 @@
 
     @Override
     public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx,
-            IRecordDescriptorProvider recordDescProvider, final int partition, int nPartitions) {
+            IRecordDescriptorProvider recordDescProvider, final int partition, int nPartitions)
+            throws HyracksDataException {
         return new AbstractUnaryInputUnaryOutputOperatorNodePushable() {
             private ByteBuffer frame = ctx.allocateFrame();
             private boolean complete = false;
diff --git a/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/base/IConfigurationFactory.java b/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/base/IConfigurationFactory.java
index dc9698b..0f41568 100644
--- a/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/base/IConfigurationFactory.java
+++ b/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/base/IConfigurationFactory.java
@@ -25,4 +25,6 @@
 
     public Configuration createConfiguration(IHyracksTaskContext ctx) throws HyracksDataException;
 
+    public Configuration createConfiguration() throws HyracksDataException;
+
 }
diff --git a/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/context/NoBudgetIndexLifecycleManager.java b/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/context/NoBudgetIndexLifecycleManager.java
new file mode 100644
index 0000000..5484f78
--- /dev/null
+++ b/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/context/NoBudgetIndexLifecycleManager.java
@@ -0,0 +1,194 @@
+/*
+ * 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.pregelix.dataflow.context;
+
+import java.io.IOException;
+import java.io.OutputStream;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.lifecycle.ILifeCycleComponent;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndex;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexLifecycleManager;
+
+public class NoBudgetIndexLifecycleManager implements IIndexLifecycleManager, ILifeCycleComponent {
+
+    private final Map<Long, IndexInfo> indexInfos;
+
+    public NoBudgetIndexLifecycleManager() {
+        this.indexInfos = new HashMap<Long, IndexInfo>();
+    }
+
+    @Override
+    public IIndex getIndex(long resourceID) {
+        IndexInfo info = indexInfos.get(resourceID);
+        return info == null ? null : info.index;
+    }
+
+    @Override
+    public void register(long resourceID, IIndex index) throws HyracksDataException {
+        if (indexInfos.containsKey(resourceID)) {
+            throw new HyracksDataException("Index with resource ID " + resourceID + " already exists.");
+        }
+
+        indexInfos.put(resourceID, new IndexInfo(index));
+    }
+
+    @Override
+    public void unregister(long resourceID) throws HyracksDataException {
+        IndexInfo info = indexInfos.remove(resourceID);
+        if (info == null) {
+            throw new HyracksDataException("Index with resource ID " + resourceID + " does not exist.");
+        }
+
+        if (info.referenceCount != 0) {
+            indexInfos.put(resourceID, info);
+            throw new HyracksDataException("Cannot remove index while it is open.");
+        }
+
+        if (info.isOpen) {
+            info.index.deactivate();
+        }
+    }
+
+    @Override
+    public void open(long resourceID) throws HyracksDataException {
+        IndexInfo info = indexInfos.get(resourceID);
+        if (info == null) {
+            throw new HyracksDataException("Failed to open index with resource ID " + resourceID
+                    + " since it does not exist.");
+        }
+
+        if (!info.isOpen) {
+            info.index.activate();
+            info.isOpen = true;
+        }
+        info.touch();
+    }
+
+    @Override
+    public void close(long resourceID) {
+        indexInfos.get(resourceID).untouch();
+    }
+
+    private class IndexInfo implements Comparable<IndexInfo> {
+        private final IIndex index;
+        private int referenceCount;
+        private long lastAccess;
+        private boolean isOpen;
+
+        public IndexInfo(IIndex index) {
+            this.index = index;
+            this.lastAccess = -1;
+            this.referenceCount = 0;
+            this.isOpen = false;
+        }
+
+        public void touch() {
+            lastAccess = System.currentTimeMillis();
+            referenceCount++;
+        }
+
+        public void untouch() {
+            lastAccess = System.currentTimeMillis();
+            referenceCount--;
+        }
+
+        @Override
+        public int compareTo(IndexInfo i) {
+            // sort by (isOpen, referenceCount, lastAccess) ascending, where
+            // true < false
+            //
+            // Example sort order:
+            // -------------------
+            // (F, 0, 70) <-- largest
+            // (F, 0, 60)
+            // (T, 10, 80)
+            // (T, 10, 70)
+            // (T, 9, 90)
+            // (T, 0, 100) <-- smallest
+            if (isOpen && !i.isOpen) {
+                return -1;
+            } else if (!isOpen && i.isOpen) {
+                return 1;
+            } else {
+                if (referenceCount < i.referenceCount) {
+                    return -1;
+                } else if (referenceCount > i.referenceCount) {
+                    return 1;
+                } else {
+                    if (lastAccess < i.lastAccess) {
+                        return -1;
+                    } else if (lastAccess > i.lastAccess) {
+                        return 1;
+                    } else {
+                        return 0;
+                    }
+                }
+            }
+
+        }
+
+        public String toString() {
+            return "{index: " + index + ", isOpen: " + isOpen + ", refCount: " + referenceCount + ", lastAccess: "
+                    + lastAccess + "}";
+        }
+    }
+
+    @Override
+    public List<IIndex> getOpenIndexes() {
+        List<IIndex> openIndexes = new ArrayList<IIndex>();
+        for (IndexInfo i : indexInfos.values()) {
+            if (i.isOpen) {
+                openIndexes.add(i.index);
+            }
+        }
+        return openIndexes;
+    }
+
+    @Override
+    public void start() {
+    }
+
+    @Override
+    public void stop(boolean dumpState, OutputStream outputStream) throws IOException {
+        if (dumpState) {
+            dumpState(outputStream);
+        }
+
+        for (IndexInfo i : indexInfos.values()) {
+            if (i.isOpen) {
+                i.index.deactivate();
+            }
+        }
+    }
+
+    private void dumpState(OutputStream os) throws IOException {
+        StringBuilder sb = new StringBuilder();
+
+        String headerFormat = "%-20s %-10s %-20s %-20s %-20s\n";
+        String rowFormat = "%-20d %-10b %-20d %-20s %-20s\n";
+        sb.append(String.format(headerFormat, "ResourceID", "Open", "Reference Count", "Last Access", "Index Name"));
+        IndexInfo ii;
+        for (Map.Entry<Long, IndexInfo> entry : indexInfos.entrySet()) {
+            ii = entry.getValue();
+            sb.append(String.format(rowFormat, entry.getKey(), ii.isOpen, ii.referenceCount, ii.lastAccess, ii.index));
+        }
+        os.write(sb.toString().getBytes());
+    }
+}
\ No newline at end of file
diff --git a/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/context/RuntimeContext.java b/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/context/RuntimeContext.java
index b86691c..24a0a9e 100644
--- a/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/context/RuntimeContext.java
+++ b/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/context/RuntimeContext.java
@@ -30,7 +30,9 @@
 import edu.uci.ics.hyracks.api.io.IWorkspaceFileFactory;
 import edu.uci.ics.hyracks.control.nc.io.IOManager;
 import edu.uci.ics.hyracks.storage.am.common.api.IIndexLifecycleManager;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.IndexLifecycleManager;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.IVirtualBufferCache;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.MultitenantVirtualBufferCache;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.VirtualBufferCache;
 import edu.uci.ics.hyracks.storage.common.buffercache.BufferCache;
 import edu.uci.ics.hyracks.storage.common.buffercache.ClockPageReplacementStrategy;
 import edu.uci.ics.hyracks.storage.common.buffercache.HeapBufferAllocator;
@@ -52,6 +54,7 @@
     private final ILocalResourceRepository localResourceRepository;
     private final ResourceIdFactory resourceIdFactory;
     private final IBufferCache bufferCache;
+    private final IVirtualBufferCache vBufferCache;
     private final IFileMapManager fileMapManager;
     private final Map<StateKey, IStateObject> appStateMap = new ConcurrentHashMap<StateKey, IStateObject>();
     private final Map<String, Long> giraphJobIdToSuperStep = new ConcurrentHashMap<String, Long>();
@@ -76,8 +79,11 @@
         bufferCache = new BufferCache(appCtx.getRootContext().getIOManager(), allocator, prs,
                 new PreDelayPageCleanerPolicy(Long.MAX_VALUE), fileMapManager, pageSize, numPages, 1000000,
                 threadFactory);
+        int numPagesInMemComponents = numPages / 4;
+        vBufferCache = new MultitenantVirtualBufferCache(new VirtualBufferCache(new HeapBufferAllocator(), pageSize,
+                numPagesInMemComponents));
         ioManager = (IOManager) appCtx.getRootContext().getIOManager();
-        lcManager = new IndexLifecycleManager();
+        lcManager = new NoBudgetIndexLifecycleManager();
         localResourceRepository = new TransientLocalResourceRepository();
         resourceIdFactory = new ResourceIdFactory(0);
     }
@@ -110,6 +116,10 @@
         return bufferCache;
     }
 
+    public IVirtualBufferCache getVirtualBufferCache() {
+        return vBufferCache;
+    }
+
     public IFileMapProvider getFileMapManager() {
         return fileMapManager;
     }
diff --git a/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/jobgen/JobGenerator.java b/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/jobgen/JobGenerator.java
index 69c5612..15117a1 100644
--- a/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/jobgen/JobGenerator.java
+++ b/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/jobgen/JobGenerator.java
@@ -23,6 +23,7 @@
 import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
 
 import edu.uci.ics.pregelix.api.job.PregelixJob;
+import edu.uci.ics.pregelix.api.util.DefaultVertexPartitioner;
 import edu.uci.ics.pregelix.example.ConnectedComponentsVertex;
 import edu.uci.ics.pregelix.example.ConnectedComponentsVertex.SimpleConnectedComponentsVertexOutputFormat;
 import edu.uci.ics.pregelix.example.GraphMutationVertex;
@@ -80,6 +81,7 @@
         job.setVertexOutputFormatClass(SimplePageRankVertexOutputFormat.class);
         job.setMessageCombinerClass(PageRankVertex.SimpleSumCombiner.class);
         job.setNoramlizedKeyComputerClass(VLongNormalizedKeyComputer.class);
+        job.setVertexPartitionerClass(DefaultVertexPartitioner.class);
         FileInputFormat.setInputPaths(job, HDFS_INPUTPATH2);
         FileOutputFormat.setOutputPath(job, new Path(HDFS_OUTPUTPAH2));
         job.getConfiguration().setLong(PregelixJob.NUM_VERTICE, 23);
@@ -132,6 +134,7 @@
         job.setVertexOutputFormatClass(SimpleConnectedComponentsVertexOutputFormat.class);
         job.setMessageCombinerClass(ConnectedComponentsVertex.SimpleMinCombiner.class);
         job.setNoramlizedKeyComputerClass(VLongNormalizedKeyComputer.class);
+        job.setVertexPartitionerClass(DefaultVertexPartitioner.class);
         FileInputFormat.setInputPaths(job, HDFS_INPUTPATH2);
         FileOutputFormat.setOutputPath(job, new Path(HDFS_OUTPUTPAH2));
         job.getConfiguration().setLong(PregelixJob.NUM_VERTICE, 23);
@@ -230,11 +233,12 @@
         job.setVertexInputFormatClass(TextMaximalCliqueInputFormat.class);
         job.setVertexOutputFormatClass(MaximalCliqueVertexOutputFormat.class);
         job.setNoramlizedKeyComputerClass(VLongNormalizedKeyComputer.class);
+        job.setMutationOrVariableSizedUpdateHeavy(true);
         FileInputFormat.setInputPaths(job, HDFS_INPUTPATH3);
         FileOutputFormat.setOutputPath(job, new Path(HDFS_OUTPUTPAH3));
         job.getConfiguration().writeXml(new FileOutputStream(new File(outputPath)));
     }
-    
+
     private static void generateMaximalCliqueJob2(String jobName, String outputPath) throws IOException {
         PregelixJob job = new PregelixJob(jobName);
         job.setVertexClass(MaximalCliqueVertex.class);
@@ -243,11 +247,12 @@
         job.setVertexInputFormatClass(TextMaximalCliqueInputFormat.class);
         job.setVertexOutputFormatClass(MaximalCliqueVertexOutputFormat.class);
         job.setNoramlizedKeyComputerClass(VLongNormalizedKeyComputer.class);
+        job.setMutationOrVariableSizedUpdateHeavy(true);
         FileInputFormat.setInputPaths(job, HDFS_INPUTPATH4);
         FileOutputFormat.setOutputPath(job, new Path(HDFS_OUTPUTPAH3));
         job.getConfiguration().writeXml(new FileOutputStream(new File(outputPath)));
     }
-    
+
     private static void generateMaximalCliqueJob3(String jobName, String outputPath) throws IOException {
         PregelixJob job = new PregelixJob(jobName);
         job.setVertexClass(MaximalCliqueVertex.class);
@@ -256,6 +261,8 @@
         job.setVertexInputFormatClass(TextMaximalCliqueInputFormat.class);
         job.setVertexOutputFormatClass(MaximalCliqueVertexOutputFormat.class);
         job.setNoramlizedKeyComputerClass(VLongNormalizedKeyComputer.class);
+        job.setVertexPartitionerClass(DefaultVertexPartitioner.class);
+        job.setMutationOrVariableSizedUpdateHeavy(true);
         FileInputFormat.setInputPaths(job, HDFS_INPUTPATH5);
         FileOutputFormat.setOutputPath(job, new Path(HDFS_OUTPUTPAH3));
         job.getConfiguration().writeXml(new FileOutputStream(new File(outputPath)));
diff --git a/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/jobrun/RunJobTestCase.java b/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/jobrun/RunJobTestCase.java
index 7b0c7da..f077053 100644
--- a/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/jobrun/RunJobTestCase.java
+++ b/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/jobrun/RunJobTestCase.java
@@ -94,7 +94,7 @@
     @Test
     public void test() throws Exception {
         setUp();
-        Plan[] plans = new Plan[] { Plan.OUTER_JOIN_SORT, Plan.OUTER_JOIN, Plan.INNER_JOIN, Plan.OUTER_JOIN_SINGLE_SORT };
+        Plan[] plans = new Plan[] { Plan.INNER_JOIN, Plan.OUTER_JOIN, Plan.OUTER_JOIN_SINGLE_SORT, Plan.OUTER_JOIN_SORT };
         for (Plan plan : plans) {
             driver.runJob(job, plan, PregelixHyracksIntegrationUtil.CC_HOST,
                     PregelixHyracksIntegrationUtil.TEST_HYRACKS_CC_CLIENT_PORT, false);
diff --git a/pregelix/pregelix-example/src/test/resources/jobs/ConnectedComponentsReal.xml b/pregelix/pregelix-example/src/test/resources/jobs/ConnectedComponentsReal.xml
index 199efab..decbde8 100644
--- a/pregelix/pregelix-example/src/test/resources/jobs/ConnectedComponentsReal.xml
+++ b/pregelix/pregelix-example/src/test/resources/jobs/ConnectedComponentsReal.xml
@@ -1,19 +1,4 @@
-<?xml version="1.0" encoding="UTF-8" standalone="no"?>
-<!--
- ! 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.
- !-->
-<configuration>
+<?xml version="1.0" encoding="UTF-8" standalone="no"?><configuration>
 <property><name>mapred.tasktracker.dns.nameserver</name><value>default</value></property>
 <property><name>mapred.queue.default.acl-administer-jobs</name><value>*</value></property>
 <property><name>mapred.skip.map.auto.incr.proc.count</name><value>true</value></property>
diff --git a/pregelix/pregelix-example/src/test/resources/jobs/ConnectedComponentsRealComplex.xml b/pregelix/pregelix-example/src/test/resources/jobs/ConnectedComponentsRealComplex.xml
index 46444b3..cca66bb 100644
--- a/pregelix/pregelix-example/src/test/resources/jobs/ConnectedComponentsRealComplex.xml
+++ b/pregelix/pregelix-example/src/test/resources/jobs/ConnectedComponentsRealComplex.xml
@@ -1,19 +1,4 @@
-<?xml version="1.0" encoding="UTF-8" standalone="no"?>
-<!--
- ! 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.
- !-->
-<configuration>
+<?xml version="1.0" encoding="UTF-8" standalone="no"?><configuration>
 <property><name>mapred.tasktracker.dns.nameserver</name><value>default</value></property>
 <property><name>mapred.queue.default.acl-administer-jobs</name><value>*</value></property>
 <property><name>mapred.skip.map.auto.incr.proc.count</name><value>true</value></property>
@@ -137,6 +122,7 @@
 <property><name>pregelix.vertexOutputFormatClass</name><value>edu.uci.ics.pregelix.example.ConnectedComponentsVertex$SimpleConnectedComponentsVertexOutputFormat</value></property>
 <property><name>mapred.reduce.copy.backoff</name><value>300</value></property>
 <property><name>mapred.map.tasks.speculative.execution</name><value>true</value></property>
+<property><name>pregelix.partitionerClass</name><value>edu.uci.ics.pregelix.api.util.DefaultVertexPartitioner</value></property>
 <property><name>mapred.inmem.merge.threshold</name><value>1000</value></property>
 <property><name>hadoop.logfile.size</name><value>10000000</value></property>
 <property><name>pregelix.vertexInputFormatClass</name><value>edu.uci.ics.pregelix.example.inputformat.TextConnectedComponentsInputFormat</value></property>
diff --git a/pregelix/pregelix-example/src/test/resources/jobs/GraphMutation.xml b/pregelix/pregelix-example/src/test/resources/jobs/GraphMutation.xml
index 47db410..d5ec8f1 100644
--- a/pregelix/pregelix-example/src/test/resources/jobs/GraphMutation.xml
+++ b/pregelix/pregelix-example/src/test/resources/jobs/GraphMutation.xml
@@ -1,19 +1,4 @@
-<?xml version="1.0" encoding="UTF-8" standalone="no"?>
-<!--
- ! 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.
- !-->
-<configuration>
+<?xml version="1.0" encoding="UTF-8" standalone="no"?><configuration>
 <property><name>mapred.tasktracker.dns.nameserver</name><value>default</value></property>
 <property><name>mapred.queue.default.acl-administer-jobs</name><value>*</value></property>
 <property><name>mapred.skip.map.auto.incr.proc.count</name><value>true</value></property>
diff --git a/pregelix/pregelix-example/src/test/resources/jobs/MaximalClique.xml b/pregelix/pregelix-example/src/test/resources/jobs/MaximalClique.xml
index c08853b..b4c42e6 100644
--- a/pregelix/pregelix-example/src/test/resources/jobs/MaximalClique.xml
+++ b/pregelix/pregelix-example/src/test/resources/jobs/MaximalClique.xml
@@ -1,19 +1,4 @@
-<?xml version="1.0" encoding="UTF-8" standalone="no"?>
-<!--
- ! 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.
- !-->
-<configuration>
+<?xml version="1.0" encoding="UTF-8" standalone="no"?><configuration>
 <property><name>mapred.tasktracker.dns.nameserver</name><value>default</value></property>
 <property><name>mapred.queue.default.acl-administer-jobs</name><value>*</value></property>
 <property><name>mapred.skip.map.auto.incr.proc.count</name><value>true</value></property>
@@ -95,6 +80,7 @@
 <property><name>mapred.tasktracker.reduce.tasks.maximum</name><value>2</value></property>
 <property><name>io.compression.codecs</name><value>org.apache.hadoop.io.compress.DefaultCodec,org.apache.hadoop.io.compress.GzipCodec,org.apache.hadoop.io.compress.BZip2Codec</value></property>
 <property><name>mapred.job.shuffle.input.buffer.percent</name><value>0.70</value></property>
+<property><name>pregelix.updateIntensive</name><value>true</value></property>
 <property><name>io.seqfile.compress.blocksize</name><value>1000000</value></property>
 <property><name>mapred.queue.names</name><value>default</value></property>
 <property><name>fs.har.impl</name><value>org.apache.hadoop.fs.HarFileSystem</value></property>
diff --git a/pregelix/pregelix-example/src/test/resources/jobs/MaximalClique2.xml b/pregelix/pregelix-example/src/test/resources/jobs/MaximalClique2.xml
index aafabee..6cf075b 100644
--- a/pregelix/pregelix-example/src/test/resources/jobs/MaximalClique2.xml
+++ b/pregelix/pregelix-example/src/test/resources/jobs/MaximalClique2.xml
@@ -1,19 +1,4 @@
-<?xml version="1.0" encoding="UTF-8" standalone="no"?>
-<!--
- ! 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.
- !-->
-<configuration>
+<?xml version="1.0" encoding="UTF-8" standalone="no"?><configuration>
 <property><name>mapred.tasktracker.dns.nameserver</name><value>default</value></property>
 <property><name>mapred.queue.default.acl-administer-jobs</name><value>*</value></property>
 <property><name>mapred.skip.map.auto.incr.proc.count</name><value>true</value></property>
@@ -95,6 +80,7 @@
 <property><name>mapred.tasktracker.reduce.tasks.maximum</name><value>2</value></property>
 <property><name>io.compression.codecs</name><value>org.apache.hadoop.io.compress.DefaultCodec,org.apache.hadoop.io.compress.GzipCodec,org.apache.hadoop.io.compress.BZip2Codec</value></property>
 <property><name>mapred.job.shuffle.input.buffer.percent</name><value>0.70</value></property>
+<property><name>pregelix.updateIntensive</name><value>true</value></property>
 <property><name>io.seqfile.compress.blocksize</name><value>1000000</value></property>
 <property><name>mapred.queue.names</name><value>default</value></property>
 <property><name>fs.har.impl</name><value>org.apache.hadoop.fs.HarFileSystem</value></property>
diff --git a/pregelix/pregelix-example/src/test/resources/jobs/MaximalClique3.xml b/pregelix/pregelix-example/src/test/resources/jobs/MaximalClique3.xml
index ee13335..49e2e6f 100644
--- a/pregelix/pregelix-example/src/test/resources/jobs/MaximalClique3.xml
+++ b/pregelix/pregelix-example/src/test/resources/jobs/MaximalClique3.xml
@@ -1,19 +1,4 @@
-<?xml version="1.0" encoding="UTF-8" standalone="no"?>
-<!--
- ! 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.
- !-->
-<configuration>
+<?xml version="1.0" encoding="UTF-8" standalone="no"?><configuration>
 <property><name>mapred.tasktracker.dns.nameserver</name><value>default</value></property>
 <property><name>mapred.queue.default.acl-administer-jobs</name><value>*</value></property>
 <property><name>mapred.skip.map.auto.incr.proc.count</name><value>true</value></property>
@@ -95,6 +80,7 @@
 <property><name>mapred.tasktracker.reduce.tasks.maximum</name><value>2</value></property>
 <property><name>io.compression.codecs</name><value>org.apache.hadoop.io.compress.DefaultCodec,org.apache.hadoop.io.compress.GzipCodec,org.apache.hadoop.io.compress.BZip2Codec</value></property>
 <property><name>mapred.job.shuffle.input.buffer.percent</name><value>0.70</value></property>
+<property><name>pregelix.updateIntensive</name><value>true</value></property>
 <property><name>io.seqfile.compress.blocksize</name><value>1000000</value></property>
 <property><name>mapred.queue.names</name><value>default</value></property>
 <property><name>fs.har.impl</name><value>org.apache.hadoop.fs.HarFileSystem</value></property>
@@ -135,6 +121,7 @@
 <property><name>pregelix.vertexOutputFormatClass</name><value>edu.uci.ics.pregelix.example.maximalclique.MaximalCliqueVertex$MaximalCliqueVertexOutputFormat</value></property>
 <property><name>mapred.reduce.copy.backoff</name><value>300</value></property>
 <property><name>mapred.map.tasks.speculative.execution</name><value>true</value></property>
+<property><name>pregelix.partitionerClass</name><value>edu.uci.ics.pregelix.api.util.DefaultVertexPartitioner</value></property>
 <property><name>mapred.inmem.merge.threshold</name><value>1000</value></property>
 <property><name>hadoop.logfile.size</name><value>10000000</value></property>
 <property><name>pregelix.vertexInputFormatClass</name><value>edu.uci.ics.pregelix.example.maximalclique.TextMaximalCliqueInputFormat</value></property>
diff --git a/pregelix/pregelix-example/src/test/resources/jobs/PageRank.xml b/pregelix/pregelix-example/src/test/resources/jobs/PageRank.xml
index 5f03027..65e0b30 100644
--- a/pregelix/pregelix-example/src/test/resources/jobs/PageRank.xml
+++ b/pregelix/pregelix-example/src/test/resources/jobs/PageRank.xml
@@ -1,19 +1,4 @@
-<?xml version="1.0" encoding="UTF-8" standalone="no"?>
-<!--
- ! 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.
- !-->
-<configuration>
+<?xml version="1.0" encoding="UTF-8" standalone="no"?><configuration>
 <property><name>mapred.tasktracker.dns.nameserver</name><value>default</value></property>
 <property><name>mapred.queue.default.acl-administer-jobs</name><value>*</value></property>
 <property><name>mapred.skip.map.auto.incr.proc.count</name><value>true</value></property>
diff --git a/pregelix/pregelix-example/src/test/resources/jobs/PageRankReal.xml b/pregelix/pregelix-example/src/test/resources/jobs/PageRankReal.xml
index 9ab92ff..9e1e0b0 100644
--- a/pregelix/pregelix-example/src/test/resources/jobs/PageRankReal.xml
+++ b/pregelix/pregelix-example/src/test/resources/jobs/PageRankReal.xml
@@ -1,19 +1,4 @@
-<?xml version="1.0" encoding="UTF-8" standalone="no"?>
-<!--
- ! 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.
- !-->
-<configuration>
+<?xml version="1.0" encoding="UTF-8" standalone="no"?><configuration>
 <property><name>mapred.tasktracker.dns.nameserver</name><value>default</value></property>
 <property><name>mapred.queue.default.acl-administer-jobs</name><value>*</value></property>
 <property><name>mapred.skip.map.auto.incr.proc.count</name><value>true</value></property>
diff --git a/pregelix/pregelix-example/src/test/resources/jobs/PageRankRealComplex.xml b/pregelix/pregelix-example/src/test/resources/jobs/PageRankRealComplex.xml
index 314ca55..c4366d7 100644
--- a/pregelix/pregelix-example/src/test/resources/jobs/PageRankRealComplex.xml
+++ b/pregelix/pregelix-example/src/test/resources/jobs/PageRankRealComplex.xml
@@ -1,19 +1,4 @@
-<?xml version="1.0" encoding="UTF-8" standalone="no"?>
-<!--
- ! 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.
- !-->
-<configuration>
+<?xml version="1.0" encoding="UTF-8" standalone="no"?><configuration>
 <property><name>mapred.tasktracker.dns.nameserver</name><value>default</value></property>
 <property><name>mapred.queue.default.acl-administer-jobs</name><value>*</value></property>
 <property><name>mapred.skip.map.auto.incr.proc.count</name><value>true</value></property>
@@ -137,6 +122,7 @@
 <property><name>pregelix.vertexOutputFormatClass</name><value>edu.uci.ics.pregelix.example.PageRankVertex$SimplePageRankVertexOutputFormat</value></property>
 <property><name>mapred.reduce.copy.backoff</name><value>300</value></property>
 <property><name>mapred.map.tasks.speculative.execution</name><value>true</value></property>
+<property><name>pregelix.partitionerClass</name><value>edu.uci.ics.pregelix.api.util.DefaultVertexPartitioner</value></property>
 <property><name>mapred.inmem.merge.threshold</name><value>1000</value></property>
 <property><name>hadoop.logfile.size</name><value>10000000</value></property>
 <property><name>pregelix.vertexInputFormatClass</name><value>edu.uci.ics.pregelix.example.inputformat.TextPageRankInputFormat</value></property>
diff --git a/pregelix/pregelix-example/src/test/resources/jobs/PageRankRealDynamic.xml b/pregelix/pregelix-example/src/test/resources/jobs/PageRankRealDynamic.xml
index f8a2055..c05a4da 100644
--- a/pregelix/pregelix-example/src/test/resources/jobs/PageRankRealDynamic.xml
+++ b/pregelix/pregelix-example/src/test/resources/jobs/PageRankRealDynamic.xml
@@ -1,19 +1,4 @@
-<?xml version="1.0" encoding="UTF-8" standalone="no"?>
-<!--
- ! 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.
- !-->
-<configuration>
+<?xml version="1.0" encoding="UTF-8" standalone="no"?><configuration>
 <property><name>mapred.tasktracker.dns.nameserver</name><value>default</value></property>
 <property><name>mapred.queue.default.acl-administer-jobs</name><value>*</value></property>
 <property><name>mapred.skip.map.auto.incr.proc.count</name><value>true</value></property>
diff --git a/pregelix/pregelix-example/src/test/resources/jobs/PageRankRealNoCombiner.xml b/pregelix/pregelix-example/src/test/resources/jobs/PageRankRealNoCombiner.xml
index f1c72fa..ac0d508 100644
--- a/pregelix/pregelix-example/src/test/resources/jobs/PageRankRealNoCombiner.xml
+++ b/pregelix/pregelix-example/src/test/resources/jobs/PageRankRealNoCombiner.xml
@@ -1,19 +1,4 @@
-<?xml version="1.0" encoding="UTF-8" standalone="no"?>
-<!--
- ! 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.
- !-->
-<configuration>
+<?xml version="1.0" encoding="UTF-8" standalone="no"?><configuration>
 <property><name>mapred.tasktracker.dns.nameserver</name><value>default</value></property>
 <property><name>mapred.queue.default.acl-administer-jobs</name><value>*</value></property>
 <property><name>mapred.skip.map.auto.incr.proc.count</name><value>true</value></property>
diff --git a/pregelix/pregelix-example/src/test/resources/jobs/ReachibilityRealComplex.xml b/pregelix/pregelix-example/src/test/resources/jobs/ReachibilityRealComplex.xml
index 427c0ea..225429a 100644
--- a/pregelix/pregelix-example/src/test/resources/jobs/ReachibilityRealComplex.xml
+++ b/pregelix/pregelix-example/src/test/resources/jobs/ReachibilityRealComplex.xml
@@ -1,19 +1,4 @@
-<?xml version="1.0" encoding="UTF-8" standalone="no"?>
-<!--
- ! 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.
- !-->
-<configuration>
+<?xml version="1.0" encoding="UTF-8" standalone="no"?><configuration>
 <property><name>mapred.tasktracker.dns.nameserver</name><value>default</value></property>
 <property><name>mapred.queue.default.acl-administer-jobs</name><value>*</value></property>
 <property><name>mapred.skip.map.auto.incr.proc.count</name><value>true</value></property>
diff --git a/pregelix/pregelix-example/src/test/resources/jobs/ReachibilityRealComplexNoConnectivity.xml b/pregelix/pregelix-example/src/test/resources/jobs/ReachibilityRealComplexNoConnectivity.xml
index 7e96163..bd9da92 100644
--- a/pregelix/pregelix-example/src/test/resources/jobs/ReachibilityRealComplexNoConnectivity.xml
+++ b/pregelix/pregelix-example/src/test/resources/jobs/ReachibilityRealComplexNoConnectivity.xml
@@ -1,19 +1,4 @@
-<?xml version="1.0" encoding="UTF-8" standalone="no"?>
-<!--
- ! 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.
- !-->
-<configuration>
+<?xml version="1.0" encoding="UTF-8" standalone="no"?><configuration>
 <property><name>mapred.tasktracker.dns.nameserver</name><value>default</value></property>
 <property><name>mapred.queue.default.acl-administer-jobs</name><value>*</value></property>
 <property><name>mapred.skip.map.auto.incr.proc.count</name><value>true</value></property>
diff --git a/pregelix/pregelix-example/src/test/resources/jobs/ShortestPaths.xml b/pregelix/pregelix-example/src/test/resources/jobs/ShortestPaths.xml
index 823e8b0..9acd7bc 100644
--- a/pregelix/pregelix-example/src/test/resources/jobs/ShortestPaths.xml
+++ b/pregelix/pregelix-example/src/test/resources/jobs/ShortestPaths.xml
@@ -1,19 +1,4 @@
-<?xml version="1.0" encoding="UTF-8" standalone="no"?>
-<!--
- ! 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.
- !-->
-<configuration>
+<?xml version="1.0" encoding="UTF-8" standalone="no"?><configuration>
 <property><name>mapred.tasktracker.dns.nameserver</name><value>default</value></property>
 <property><name>mapred.queue.default.acl-administer-jobs</name><value>*</value></property>
 <property><name>mapred.skip.map.auto.incr.proc.count</name><value>true</value></property>
diff --git a/pregelix/pregelix-example/src/test/resources/jobs/ShortestPathsReal.xml b/pregelix/pregelix-example/src/test/resources/jobs/ShortestPathsReal.xml
index 4ae102b..6c25575 100644
--- a/pregelix/pregelix-example/src/test/resources/jobs/ShortestPathsReal.xml
+++ b/pregelix/pregelix-example/src/test/resources/jobs/ShortestPathsReal.xml
@@ -1,19 +1,4 @@
-<?xml version="1.0" encoding="UTF-8" standalone="no"?>
-<!--
- ! 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.
- !-->
-<configuration>
+<?xml version="1.0" encoding="UTF-8" standalone="no"?><configuration>
 <property><name>mapred.tasktracker.dns.nameserver</name><value>default</value></property>
 <property><name>mapred.queue.default.acl-administer-jobs</name><value>*</value></property>
 <property><name>mapred.skip.map.auto.incr.proc.count</name><value>true</value></property>
diff --git a/pregelix/pregelix-example/src/test/resources/jobs/TriangleCounting.xml b/pregelix/pregelix-example/src/test/resources/jobs/TriangleCounting.xml
index 228a526..4a40a6a 100644
--- a/pregelix/pregelix-example/src/test/resources/jobs/TriangleCounting.xml
+++ b/pregelix/pregelix-example/src/test/resources/jobs/TriangleCounting.xml
@@ -1,19 +1,4 @@
-<?xml version="1.0" encoding="UTF-8" standalone="no"?>
-<!--
- ! 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.
- !-->
-<configuration>
+<?xml version="1.0" encoding="UTF-8" standalone="no"?><configuration>
 <property><name>mapred.tasktracker.dns.nameserver</name><value>default</value></property>
 <property><name>mapred.queue.default.acl-administer-jobs</name><value>*</value></property>
 <property><name>mapred.skip.map.auto.incr.proc.count</name><value>true</value></property>
diff --git a/pregelix/pregelix-runtime/pom.xml b/pregelix/pregelix-runtime/pom.xml
index ae9f47e..54e2256 100644
--- a/pregelix/pregelix-runtime/pom.xml
+++ b/pregelix/pregelix-runtime/pom.xml
@@ -148,6 +148,13 @@
 		</dependency>
 		<dependency>
 			<groupId>edu.uci.ics.hyracks</groupId>
+			<artifactId>hyracks-storage-am-lsm-common</artifactId>
+			<version>0.2.7-SNAPSHOT</version>
+			<type>jar</type>
+			<scope>compile</scope>
+		</dependency>
+		<dependency>
+			<groupId>edu.uci.ics.hyracks</groupId>
 			<artifactId>hyracks-control-cc</artifactId>
 			<version>0.2.7-SNAPSHOT</version>
 			<type>jar</type>
diff --git a/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/bootstrap/VirtualBufferCacheProvider.java b/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/bootstrap/VirtualBufferCacheProvider.java
new file mode 100644
index 0000000..ec51047
--- /dev/null
+++ b/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/bootstrap/VirtualBufferCacheProvider.java
@@ -0,0 +1,39 @@
+/*
+ * 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.pregelix.runtime.bootstrap;
+
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.IVirtualBufferCache;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.IVirtualBufferCacheProvider;
+import edu.uci.ics.pregelix.dataflow.context.RuntimeContext;
+
+/**
+ * The virtual buffer cache provider
+ * 
+ * @author yingyib
+ */
+public class VirtualBufferCacheProvider implements IVirtualBufferCacheProvider {
+
+    private static final long serialVersionUID = 1L;
+
+    public VirtualBufferCacheProvider(){
+        
+    }
+    
+    @Override
+    public synchronized IVirtualBufferCache getVirtualBufferCache(IHyracksTaskContext ctx) {
+        return RuntimeContext.get(ctx).getVirtualBufferCache();
+    }
+}
diff --git a/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/function/NoOpUpdateFunctionFactory.java b/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/function/NoOpUpdateFunctionFactory.java
new file mode 100644
index 0000000..88577c2
--- /dev/null
+++ b/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/function/NoOpUpdateFunctionFactory.java
@@ -0,0 +1,67 @@
+/*
+ * 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.pregelix.runtime.function;
+
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
+import edu.uci.ics.pregelix.dataflow.std.base.IUpdateFunction;
+import edu.uci.ics.pregelix.dataflow.std.base.IUpdateFunctionFactory;
+
+/**
+ * No operation update function factory
+ * 
+ * @author yingyib
+ */
+public class NoOpUpdateFunctionFactory implements IUpdateFunctionFactory {
+    private static final long serialVersionUID = 1L;
+    public static NoOpUpdateFunctionFactory INSTANCE = new NoOpUpdateFunctionFactory();
+
+    private NoOpUpdateFunctionFactory() {
+
+    }
+
+    @Override
+    public IUpdateFunction createFunction() {
+        return new IUpdateFunction() {
+
+            @Override
+            public void open(IHyracksTaskContext ctx, RecordDescriptor rd, IFrameWriter... writer)
+                    throws HyracksDataException {
+
+            }
+
+            @Override
+            public void process(Object[] tuple) throws HyracksDataException {
+
+            }
+
+            @Override
+            public void close() throws HyracksDataException {
+
+            }
+
+            @Override
+            public void update(ITupleReference tupleRef, ArrayTupleBuilder cloneUpdateTb) throws HyracksDataException {
+
+            }
+
+        };
+    }
+
+}
diff --git a/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/touchpoint/VertexPartitionComputerFactory.java b/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/touchpoint/VertexPartitionComputerFactory.java
new file mode 100644
index 0000000..acccabb
--- /dev/null
+++ b/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/touchpoint/VertexPartitionComputerFactory.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.pregelix.runtime.touchpoint;
+
+import java.io.DataInputStream;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.WritableComparable;
+
+import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
+import edu.uci.ics.hyracks.api.dataflow.value.ITuplePartitionComputer;
+import edu.uci.ics.hyracks.api.dataflow.value.ITuplePartitionComputerFactory;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.common.comm.util.ByteBufferInputStream;
+import edu.uci.ics.pregelix.api.graph.VertexPartitioner;
+import edu.uci.ics.pregelix.api.util.BspUtils;
+import edu.uci.ics.pregelix.dataflow.base.IConfigurationFactory;
+
+/**
+ * The vertex-based partition computer factory.
+ * It is used to support customized graph partitioning function.
+ * 
+ * @author yingyib
+ */
+public class VertexPartitionComputerFactory implements ITuplePartitionComputerFactory {
+
+    private static final long serialVersionUID = 1L;
+    private final IConfigurationFactory confFactory;
+
+    public VertexPartitionComputerFactory(IConfigurationFactory confFactory) {
+        this.confFactory = confFactory;
+    }
+
+    @SuppressWarnings("rawtypes")
+    public ITuplePartitionComputer createPartitioner() {
+        try {
+            final Configuration conf = confFactory.createConfiguration();
+            return new ITuplePartitionComputer() {
+                private final ByteBufferInputStream bbis = new ByteBufferInputStream();
+                private final DataInputStream dis = new DataInputStream(bbis);
+                private final VertexPartitioner partitioner = BspUtils.createVertexPartitioner(conf);
+                private final WritableComparable vertexId = BspUtils.createVertexIndex(conf);
+
+                @SuppressWarnings("unchecked")
+                public int partition(IFrameTupleAccessor accessor, int tIndex, int nParts) throws HyracksDataException {
+                    try {
+                        int keyStart = accessor.getTupleStartOffset(tIndex) + accessor.getFieldSlotsLength()
+                                + accessor.getFieldStartOffset(tIndex, 0);
+                        bbis.setByteBuffer(accessor.getBuffer(), keyStart);
+                        vertexId.readFields(dis);
+                        return Math.abs(partitioner.getPartitionId(vertexId, nParts) % nParts);
+                    } catch (Exception e) {
+                        throw new HyracksDataException(e);
+                    }
+                }
+            };
+        } catch (Exception e) {
+            throw new RuntimeException(e);
+        }
+    }
+}